提交 0fd18a1b 编写于 作者: L Liang Zhang 提交者: Juan Pan(Trista)

Fixes #3627 (#3629)

* move StatementExecuteUnit to sql package

* move ShardingExecuteEngine to engine package

* for code style

* rename QueryResultBasedQueryResultFixture and ResultSetBasedQueryResultFixture

* refactor DistinctQueryResult.getQueryRows()

* skip distinct merge for group by and aggregation only

* add DQLMergeEngine.isNeedProcessOrderBy

* use group by to merge distinct row

* add DQLMergeEngine.setGroupByForDistinctRow

* remove useless SelectItemsSegment.findSelectItemSegments

* code style for AggregationDistinctQueryResult

* code style for AggregationDistinctQueryResult

* remove DistinctQueryResult.divide()

* remove useless DistinctQueryResult

* use group by to process aggregation distinct

* remove useless AggregationDistinctQueryResult

* remove useless AggregationDistinctQueryMetaData

* remove useless QueryResult.getQueryResultMetaData()

* remove useless QueryRow.distinctColumnIndexes

* remove useless QueryRow.distinctColumnIndexes

* remove useless QueryRow.@EqualsAndHashCode

* remove useless QueryRow

* adjust constructor seq of DQLMergeEngine

* refactor DQLMergeEngine's constructor

* move ProjectionsContext.getColumnLabels() to ProjectionsContextEngine

* remove DQLMergeEngine.tableMetas

* refactor DQLMergeEngine

* code style for ProjectionsContextEngine
上级 c111a85e
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute;
package org.apache.shardingsphere.core.execute.engine;
import java.sql.SQLException;
import java.util.Map;
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute;
package org.apache.shardingsphere.core.execute.engine;
import lombok.AccessLevel;
import lombok.NoArgsConstructor;
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute;
package org.apache.shardingsphere.core.execute.engine;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ListenableFuture;
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute;
package org.apache.shardingsphere.core.execute.engine;
import lombok.Getter;
import lombok.RequiredArgsConstructor;
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute;
package org.apache.shardingsphere.core.execute.engine;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute;
package org.apache.shardingsphere.core.execute.engine;
import java.sql.SQLException;
import java.util.Collection;
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute;
package org.apache.shardingsphere.core.execute.engine;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import lombok.AccessLevel;
......
......@@ -17,6 +17,14 @@
package org.apache.shardingsphere.core.execute.metadata;
import com.google.common.base.Optional;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteEngine;
import org.apache.shardingsphere.core.metadata.datasource.DataSourceMetas;
import org.apache.shardingsphere.core.metadata.table.TableMetaData;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.core.rule.TableRule;
import org.apache.shardingsphere.spi.database.DataSourceMetaData;
import java.sql.Connection;
import java.sql.ResultSet;
import java.sql.SQLException;
......@@ -25,15 +33,6 @@ import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.Map;
import org.apache.shardingsphere.core.execute.ShardingExecuteEngine;
import org.apache.shardingsphere.core.metadata.datasource.DataSourceMetas;
import org.apache.shardingsphere.core.metadata.table.TableMetaData;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.core.rule.TableRule;
import org.apache.shardingsphere.spi.database.DataSourceMetaData;
import com.google.common.base.Optional;
/**
* Table meta data initializer.
*
......
......@@ -21,9 +21,9 @@ import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.core.execute.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.ShardingGroupExecuteCallback;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.engine.ShardingGroupExecuteCallback;
import org.apache.shardingsphere.core.metadata.column.ColumnMetaData;
import org.apache.shardingsphere.core.metadata.column.EncryptColumnMetaData;
import org.apache.shardingsphere.core.metadata.column.ShardingGeneratedKeyColumnMetaData;
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute;
package org.apache.shardingsphere.core.execute.sql;
import lombok.Getter;
import lombok.RequiredArgsConstructor;
......
......@@ -19,8 +19,8 @@ package org.apache.shardingsphere.core.execute.sql.execute;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.ShardingGroupExecuteCallback;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingGroupExecuteCallback;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.hook.SPISQLExecutionHook;
import org.apache.shardingsphere.core.execute.hook.SQLExecutionHook;
import org.apache.shardingsphere.core.execute.sql.execute.threadlocal.ExecutorExceptionHandler;
......
......@@ -18,9 +18,9 @@
package org.apache.shardingsphere.core.execute.sql.execute;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.execute.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.execute.threadlocal.ExecutorExceptionHandler;
import java.sql.SQLException;
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute.sql.execute.result;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationDistinctProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationProjection;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import java.util.Collection;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
/**
* Aggregation distinct query metadata.
*
* @author panjuan
*/
public final class AggregationDistinctQueryMetaData {
private final Collection<AggregationDistinctColumnMetaData> aggregationDistinctColumnMetaDataList = new LinkedList<>();
private final Map<Integer, String> aggregationDistinctColumnIndexAndLabels = new HashMap<>();
private final Map<Integer, AggregationType> aggregationDistinctColumnIndexAndAggregationTypes = new HashMap<>();
private final Map<Integer, Integer> aggregationDistinctColumnIndexAndCountColumnIndexes = new HashMap<>();
private final Map<Integer, Integer> aggregationDistinctColumnIndexAndSumColumnIndexes = new HashMap<>();
public AggregationDistinctQueryMetaData(final Collection<AggregationDistinctProjection> aggregationDistinctProjections, final QueryResultMetaData queryResultMetaData) {
aggregationDistinctColumnMetaDataList.addAll(getColumnMetaDataList(aggregationDistinctProjections, queryResultMetaData));
aggregationDistinctColumnIndexAndLabels.putAll(getAggregationDistinctColumnIndexAndLabels());
aggregationDistinctColumnIndexAndAggregationTypes.putAll(getAggregationDistinctColumnIndexAndAggregationTypes());
aggregationDistinctColumnIndexAndCountColumnIndexes.putAll(getAggregationDistinctColumnIndexAndCountColumnIndexes());
aggregationDistinctColumnIndexAndSumColumnIndexes.putAll(getAggregationDistinctColumnIndexAndSumColumnIndexes());
}
private Collection<AggregationDistinctColumnMetaData> getColumnMetaDataList(final Collection<AggregationDistinctProjection> aggregationDistinctProjections,
final QueryResultMetaData queryResultMetaData) {
Collection<AggregationDistinctColumnMetaData> result = new LinkedList<>();
for (AggregationDistinctProjection each : aggregationDistinctProjections) {
result.add(getAggregationDistinctColumnMetaData(each, queryResultMetaData.getColumnIndex(each.getColumnLabel()), queryResultMetaData));
}
return result;
}
private AggregationDistinctColumnMetaData getAggregationDistinctColumnMetaData(final AggregationDistinctProjection aggregationDistinctProjection,
final int aggregationDistinctColumnIndex, final QueryResultMetaData queryResultMetaData) {
List<AggregationProjection> derivedProjections = aggregationDistinctProjection.getDerivedAggregationProjections();
if (derivedProjections.isEmpty()) {
return new AggregationDistinctColumnMetaData(aggregationDistinctColumnIndex, aggregationDistinctProjection.getColumnLabel(), aggregationDistinctProjection.getType());
}
int countDerivedIndex = queryResultMetaData.getColumnIndex(derivedProjections.get(0).getColumnLabel());
int sumDerivedIndex = queryResultMetaData.getColumnIndex(derivedProjections.get(1).getColumnLabel());
return new AggregationDistinctColumnMetaData(
aggregationDistinctColumnIndex, aggregationDistinctProjection.getColumnLabel(), aggregationDistinctProjection.getType(), countDerivedIndex, sumDerivedIndex);
}
private Map<Integer, String> getAggregationDistinctColumnIndexAndLabels() {
Map<Integer, String> result = new HashMap<>();
for (AggregationDistinctColumnMetaData each : aggregationDistinctColumnMetaDataList) {
result.put(each.columnIndex, each.columnLabel);
}
return result;
}
private Map<Integer, AggregationType> getAggregationDistinctColumnIndexAndAggregationTypes() {
Map<Integer, AggregationType> result = new LinkedHashMap<>();
for (AggregationDistinctColumnMetaData each : aggregationDistinctColumnMetaDataList) {
result.put(each.columnIndex, each.aggregationType);
}
return result;
}
private Map<Integer, Integer> getAggregationDistinctColumnIndexAndCountColumnIndexes() {
Map<Integer, Integer> result = new HashMap<>();
for (AggregationDistinctColumnMetaData each : aggregationDistinctColumnMetaDataList) {
result.put(each.columnIndex, each.derivedCountIndex);
}
return result;
}
private Map<Integer, Integer> getAggregationDistinctColumnIndexAndSumColumnIndexes() {
Map<Integer, Integer> result = new HashMap<>();
for (AggregationDistinctColumnMetaData each : aggregationDistinctColumnMetaDataList) {
result.put(each.columnIndex, each.derivedSumIndex);
}
return result;
}
/**
* Is aggregation distinct column index.
*
* @param columnIndex column index
* @return is aggregation distinct column index or not
*/
public boolean isAggregationDistinctColumnIndex(final int columnIndex) {
return aggregationDistinctColumnIndexAndLabels.keySet().contains(columnIndex);
}
/**
* Is aggregation distinct column label.
*
* @param columnLabel column label
* @return is aggregation distinct column label or not
*/
public boolean isAggregationDistinctColumnLabel(final String columnLabel) {
return aggregationDistinctColumnIndexAndLabels.values().contains(columnLabel);
}
/**
* Get aggregation type.
*
* @param distinctColumnIndex distinct column index
* @return aggregation type
*/
public AggregationType getAggregationType(final int distinctColumnIndex) {
return aggregationDistinctColumnIndexAndAggregationTypes.get(distinctColumnIndex);
}
/**
* Is derived count column index.
*
* @param columnIndex column index
* @return is derived count column index or not
*/
public boolean isDerivedCountColumnIndex(final int columnIndex) {
return aggregationDistinctColumnIndexAndCountColumnIndexes.values().contains(columnIndex);
}
/**
* Is derived sum column index.
*
* @param columnIndex column index
* @return is derived sum column index or not
*/
public boolean isDerivedSumColumnIndex(final int columnIndex) {
return aggregationDistinctColumnIndexAndSumColumnIndexes.values().contains(columnIndex);
}
/**
* Get aggregation distinct column index.
*
* @param derivedSumIndex derived sum index
* @return aggregation distinct column index
*/
public int getAggregationDistinctColumnIndex(final int derivedSumIndex) {
for (Entry<Integer, Integer> entry : aggregationDistinctColumnIndexAndSumColumnIndexes.entrySet()) {
if (entry.getValue().equals(derivedSumIndex)) {
return entry.getKey();
}
}
throw new ShardingException("Can not get aggregation distinct column index.");
}
/**
* Get aggregation distinct column index.
*
* @param distinctColumnLabel aggregation distinct column label
* @return aggregation distinct column index
*/
public int getAggregationDistinctColumnIndex(final String distinctColumnLabel) {
for (Entry<Integer, String> entry : aggregationDistinctColumnIndexAndLabels.entrySet()) {
if (entry.getValue().equals(distinctColumnLabel)) {
return entry.getKey();
}
}
throw new ShardingException("Can not get aggregation distinct column index.");
}
/**
* Get aggregation distinct column label.
*
* @param distinctColumnIndex aggregation distinct column index
* @return aggregation distinct column label
*/
public String getAggregationDistinctColumnLabel(final int distinctColumnIndex) {
return aggregationDistinctColumnIndexAndLabels.get(distinctColumnIndex);
}
@RequiredArgsConstructor
private final class AggregationDistinctColumnMetaData {
private final int columnIndex;
private final String columnLabel;
private final AggregationType aggregationType;
private final int derivedCountIndex;
private final int derivedSumIndex;
private AggregationDistinctColumnMetaData(final int columnIndex, final String columnLabel, final AggregationType aggregationType) {
this(columnIndex, columnLabel, aggregationType, -1, -1);
}
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute.sql.execute.result;
import com.google.common.base.Function;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import org.apache.shardingsphere.core.execute.sql.execute.row.QueryRow;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationDistinctProjection;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import java.io.InputStream;
import java.sql.SQLException;
import java.util.Calendar;
import java.util.Collection;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
/**
* Aggregation distinct query result.
*
* @author panjuan
*/
public final class AggregationDistinctQueryResult extends DistinctQueryResult {
private final AggregationDistinctQueryMetaData metaData;
private AggregationDistinctQueryResult(final QueryResultMetaData queryResultMetaData, final Iterator<QueryRow> resultData, final AggregationDistinctQueryMetaData distinctQueryMetaData) {
super(queryResultMetaData, resultData);
metaData = distinctQueryMetaData;
}
public AggregationDistinctQueryResult(final Collection<QueryResult> queryResults, final List<AggregationDistinctProjection> aggregationDistinctProjections) throws SQLException {
super(queryResults, Lists.transform(aggregationDistinctProjections, new Function<AggregationDistinctProjection, String>() {
@Override
public String apply(final AggregationDistinctProjection input) {
return input.getDistinctColumnLabel();
}
}));
metaData = new AggregationDistinctQueryMetaData(aggregationDistinctProjections, getQueryResultMetaData());
}
/**
* Divide one distinct query result to multiple child ones.
*
* @return multiple child distinct query results
*/
@Override
public List<DistinctQueryResult> divide() {
return Lists.newArrayList(Iterators.transform(getResultData(), new Function<QueryRow, DistinctQueryResult>() {
@Override
public DistinctQueryResult apply(final QueryRow input) {
Set<QueryRow> resultData = new LinkedHashSet<>();
resultData.add(input);
return new AggregationDistinctQueryResult(getQueryResultMetaData(), resultData.iterator(), metaData);
}
}));
}
private Object getValue(final int columnIndex) {
if (metaData.isAggregationDistinctColumnIndex(columnIndex)) {
return AggregationType.COUNT == metaData.getAggregationType(columnIndex) ? 1 : super.getValue(columnIndex, Object.class);
}
if (metaData.isDerivedCountColumnIndex(columnIndex)) {
return 1;
}
if (metaData.isDerivedSumColumnIndex(columnIndex)) {
return super.getValue(metaData.getAggregationDistinctColumnIndex(columnIndex), Object.class);
}
return super.getValue(columnIndex, Object.class);
}
private Object getValue(final String columnLabel) {
return getValue(getColumnIndex(columnLabel));
}
@Override
public Object getValue(final int columnIndex, final Class<?> type) {
return getValue(columnIndex);
}
@Override
public Object getValue(final String columnLabel, final Class<?> type) {
return getValue(columnLabel);
}
@Override
public Object getCalendarValue(final int columnIndex, final Class<?> type, final Calendar calendar) {
return getValue(columnIndex);
}
@Override
public Object getCalendarValue(final String columnLabel, final Class<?> type, final Calendar calendar) {
return getValue(columnLabel);
}
@Override
public InputStream getInputStream(final int columnIndex, final String type) {
return getInputStream(getValue(columnIndex));
}
@Override
public InputStream getInputStream(final String columnLabel, final String type) {
return getInputStream(getValue(columnLabel));
}
@Override
public boolean wasNull() {
return null == getCurrentRow();
}
@Override
public String getColumnLabel(final int columnIndex) throws SQLException {
if (metaData.isAggregationDistinctColumnIndex(columnIndex)) {
return metaData.getAggregationDistinctColumnLabel(columnIndex);
}
String columnLabel = getQueryResultMetaData().getColumnLabel(columnIndex);
if (null != columnLabel) {
return columnLabel;
}
throw new SQLException("Column index out of range", "9999");
}
@Override
protected Integer getColumnIndex(final String columnLabel) {
return isContainColumnLabel(columnLabel) ? metaData.getAggregationDistinctColumnIndex(columnLabel) : super.getColumnIndex(columnLabel);
}
private boolean isContainColumnLabel(final String columnLabel) {
return null != metaData && metaData.isAggregationDistinctColumnLabel(columnLabel);
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute.sql.execute.result;
import com.google.common.base.Function;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import lombok.AccessLevel;
import lombok.Getter;
import lombok.RequiredArgsConstructor;
import lombok.SneakyThrows;
import org.apache.shardingsphere.core.execute.sql.execute.row.QueryRow;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.InputStream;
import java.io.ObjectOutputStream;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Calendar;
import java.util.Collection;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
/**
* Distinct query result.
*
* @author panjuan
* @author yangyi
* @author sunbufu
*/
@RequiredArgsConstructor
@Getter(AccessLevel.PROTECTED)
public class DistinctQueryResult implements QueryResult {
@Getter
private final QueryResultMetaData queryResultMetaData;
private final Iterator<QueryRow> resultData;
private QueryRow currentRow;
public DistinctQueryResult(final Collection<QueryResult> queryResults, final List<String> distinctColumnLabels) throws SQLException {
QueryResult firstQueryResult = queryResults.iterator().next();
this.queryResultMetaData = firstQueryResult.getQueryResultMetaData();
resultData = getResultData(queryResults, distinctColumnLabels);
}
private Iterator<QueryRow> getResultData(final Collection<QueryResult> queryResults, final List<String> distinctColumnLabels) throws SQLException {
Set<QueryRow> result = new LinkedHashSet<>();
List<Integer> distinctColumnIndexes = Lists.transform(distinctColumnLabels, new Function<String, Integer>() {
@Override
public Integer apply(final String input) {
return getColumnIndex(input);
}
});
for (QueryResult each : queryResults) {
fill(result, each, distinctColumnIndexes);
}
return result.iterator();
}
private void fill(final Set<QueryRow> resultData, final QueryResult queryResult, final List<Integer> distinctColumnIndexes) throws SQLException {
while (queryResult.next()) {
List<Object> rowData = new ArrayList<>(queryResult.getColumnCount());
for (int columnIndex = 1; columnIndex <= queryResult.getColumnCount(); columnIndex++) {
rowData.add(queryResult.getValue(columnIndex, Object.class));
}
resultData.add(new QueryRow(rowData, distinctColumnIndexes));
}
}
/**
* Divide one distinct query result to multiple child ones.
*
* @return multiple child distinct query results
*/
public List<DistinctQueryResult> divide() {
return Lists.newArrayList(Iterators.transform(resultData, new Function<QueryRow, DistinctQueryResult>() {
@Override
public DistinctQueryResult apply(final QueryRow row) {
Set<QueryRow> resultData = new LinkedHashSet<>();
resultData.add(row);
return new DistinctQueryResult(queryResultMetaData, resultData.iterator());
}
}));
}
@Override
public final boolean next() {
if (resultData.hasNext()) {
currentRow = resultData.next();
return true;
}
currentRow = null;
return false;
}
@Override
public Object getValue(final int columnIndex, final Class<?> type) {
return currentRow.getValue(columnIndex);
}
@Override
public Object getValue(final String columnLabel, final Class<?> type) {
return currentRow.getValue(getColumnIndex(columnLabel));
}
@Override
public Object getCalendarValue(final int columnIndex, final Class<?> type, final Calendar calendar) {
return currentRow.getValue(columnIndex);
}
@Override
public Object getCalendarValue(final String columnLabel, final Class<?> type, final Calendar calendar) {
return currentRow.getValue(getColumnIndex(columnLabel));
}
@Override
public InputStream getInputStream(final int columnIndex, final String type) {
return getInputStream(currentRow.getValue(columnIndex));
}
@Override
public InputStream getInputStream(final String columnLabel, final String type) {
return getInputStream(currentRow.getValue(getColumnIndex(columnLabel)));
}
@SneakyThrows
protected InputStream getInputStream(final Object value) {
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
ObjectOutputStream objectOutputStream = new ObjectOutputStream(byteArrayOutputStream);
objectOutputStream.writeObject(value);
objectOutputStream.flush();
objectOutputStream.close();
return new ByteArrayInputStream(byteArrayOutputStream.toByteArray());
}
@Override
public boolean wasNull() {
return null == currentRow;
}
@Override
public boolean isCaseSensitive(final int columnIndex) throws SQLException {
return queryResultMetaData.isCaseSensitive(columnIndex);
}
@Override
public int getColumnCount() throws SQLException {
return queryResultMetaData.getColumnCount();
}
@Override
public String getColumnLabel(final int columnIndex) throws SQLException {
String columnLabel = queryResultMetaData.getColumnLabel(columnIndex);
if (null != columnLabel) {
return columnLabel;
}
throw new SQLException("Column index out of range", "9999");
}
protected Integer getColumnIndex(final String columnLabel) {
return queryResultMetaData.getColumnIndex(columnLabel);
}
}
......@@ -18,14 +18,12 @@
package org.apache.shardingsphere.core.execute.sql.execute.result;
import com.google.common.base.Optional;
import lombok.Getter;
import lombok.SneakyThrows;
import org.apache.shardingsphere.core.constant.properties.ShardingProperties;
import org.apache.shardingsphere.core.execute.sql.execute.row.QueryRow;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.core.rule.EncryptRule;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.spi.encrypt.ShardingEncryptor;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
......@@ -49,44 +47,43 @@ import java.util.List;
*/
public final class MemoryQueryResult implements QueryResult {
private final Iterator<QueryRow> resultData;
private final Iterator<List<Object>> rows;
private List<Object> currentRow;
private QueryRow currentRow;
@Getter
private final QueryResultMetaData queryResultMetaData;
public MemoryQueryResult(final ResultSet resultSet, final ShardingRule shardingRule, final ShardingProperties properties, final SQLStatementContext sqlStatementContext) throws SQLException {
resultData = getResultData(resultSet);
rows = getRows(resultSet);
queryResultMetaData = new QueryResultMetaData(resultSet.getMetaData(), shardingRule, properties, sqlStatementContext);
}
public MemoryQueryResult(final ResultSet resultSet, final EncryptRule encryptRule, final ShardingProperties properties, final SQLStatementContext sqlStatementContext) throws SQLException {
resultData = getResultData(resultSet);
rows = getRows(resultSet);
queryResultMetaData = new QueryResultMetaData(resultSet.getMetaData(), encryptRule, properties, sqlStatementContext);
}
public MemoryQueryResult(final ResultSet resultSet) throws SQLException {
resultData = getResultData(resultSet);
rows = getRows(resultSet);
queryResultMetaData = new QueryResultMetaData(resultSet.getMetaData());
}
private Iterator<QueryRow> getResultData(final ResultSet resultSet) throws SQLException {
Collection<QueryRow> result = new LinkedList<>();
private Iterator<List<Object>> getRows(final ResultSet resultSet) throws SQLException {
Collection<List<Object>> result = new LinkedList<>();
while (resultSet.next()) {
List<Object> rowData = new ArrayList<>(resultSet.getMetaData().getColumnCount());
for (int columnIndex = 1; columnIndex <= resultSet.getMetaData().getColumnCount(); columnIndex++) {
rowData.add(QueryResultUtil.getValue(resultSet, columnIndex));
}
result.add(new QueryRow(rowData));
result.add(rowData);
}
return result.iterator();
}
@Override
public boolean next() {
if (resultData.hasNext()) {
currentRow = resultData.next();
if (rows.hasNext()) {
currentRow = rows.next();
return true;
}
currentRow = null;
......@@ -95,32 +92,32 @@ public final class MemoryQueryResult implements QueryResult {
@Override
public Object getValue(final int columnIndex, final Class<?> type) throws SQLException {
return decrypt(columnIndex, currentRow.getValue(columnIndex));
return decrypt(columnIndex, currentRow.get(columnIndex - 1));
}
@Override
public Object getValue(final String columnLabel, final Class<?> type) throws SQLException {
return decrypt(columnLabel, currentRow.getValue(queryResultMetaData.getColumnIndex(columnLabel)));
return decrypt(columnLabel, currentRow.get(queryResultMetaData.getColumnIndex(columnLabel) - 1));
}
@Override
public Object getCalendarValue(final int columnIndex, final Class<?> type, final Calendar calendar) {
return currentRow.getValue(columnIndex);
return currentRow.get(columnIndex - 1);
}
@Override
public Object getCalendarValue(final String columnLabel, final Class<?> type, final Calendar calendar) {
return currentRow.getValue(queryResultMetaData.getColumnIndex(columnLabel));
return currentRow.get(queryResultMetaData.getColumnIndex(columnLabel) - 1);
}
@Override
public InputStream getInputStream(final int columnIndex, final String type) {
return getInputStream(currentRow.getValue(columnIndex));
return getInputStream(currentRow.get(columnIndex - 1));
}
@Override
public InputStream getInputStream(final String columnLabel, final String type) {
return getInputStream(currentRow.getValue(queryResultMetaData.getColumnIndex(columnLabel)));
return getInputStream(currentRow.get(queryResultMetaData.getColumnIndex(columnLabel) - 1));
}
@SneakyThrows
......
......@@ -132,11 +132,4 @@ public interface QueryResult {
* @throws SQLException SQL Exception
*/
boolean isCaseSensitive(int columnIndex) throws SQLException;
/**
* Get QueryResultMetaData.
*
* @return QueryResultMetaData
*/
QueryResultMetaData getQueryResultMetaData();
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute.sql.execute.row;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
import lombok.Getter;
import lombok.RequiredArgsConstructor;
import java.util.Collections;
import java.util.List;
/**
* Query row.
*
* @author panjuan
*/
@RequiredArgsConstructor
@Getter
public final class QueryRow {
private final List<Object> rowData;
private final List<Integer> distinctColumnIndexes;
public QueryRow(final List<Object> rowData) {
this(rowData, Collections.<Integer>emptyList());
}
/**
* Get value.
*
* @param columnIndex column index
* @return value
*/
public Object getValue(final int columnIndex) {
return rowData.get(columnIndex - 1);
}
@Override
public boolean equals(final Object obj) {
return this == obj || null != obj && getClass() == obj.getClass() && isEqual((QueryRow) obj);
}
private boolean isEqual(final QueryRow queryRow) {
if (distinctColumnIndexes.isEmpty()) {
return rowData.equals(queryRow.getRowData());
}
return distinctColumnIndexes.equals(queryRow.getDistinctColumnIndexes()) && isEqualPartly(queryRow);
}
private boolean isEqualPartly(final QueryRow queryRow) {
for (int i = 0; i < distinctColumnIndexes.size(); i++) {
if (null == rowData.get(i) && null != queryRow.getRowData().get(i)) {
return false;
}
if (null != rowData.get(i) && !rowData.get(i).equals(queryRow.getRowData().get(i))) {
return false;
}
}
return true;
}
@Override
public int hashCode() {
return distinctColumnIndexes.isEmpty() ? rowData.hashCode() : Lists.transform(distinctColumnIndexes, new Function<Integer, Object>() {
@Override
public Object apply(final Integer input) {
return rowData.get(input - 1);
}
}).hashCode();
}
}
......@@ -18,7 +18,7 @@
package org.apache.shardingsphere.core.execute.sql.prepare;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.route.RouteUnit;
import java.sql.Connection;
......
......@@ -20,8 +20,8 @@ package org.apache.shardingsphere.core.execute.sql.prepare;
import com.google.common.collect.Lists;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.route.RouteUnit;
import org.apache.shardingsphere.core.route.SQLUnit;
......
......@@ -15,9 +15,10 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute;
package org.apache.shardingsphere.core.execute.engine;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
......@@ -104,7 +105,7 @@ public final class ShardingExecuteEngineTest {
@RequiredArgsConstructor
private final class MockGroupExecuteCallback implements ShardingExecuteCallback<StatementExecuteUnit, String>, ShardingGroupExecuteCallback<StatementExecuteUnit, String> {
private final CountDownLatch latch;
@Override
......@@ -112,7 +113,7 @@ public final class ShardingExecuteEngineTest {
latch.countDown();
return "succeed";
}
@Override
public Collection<String> execute(final Collection<StatementExecuteUnit> inputs, final boolean isTrunkThread, final Map<String, Object> shardingExecuteDataMap) {
List<String> result = new LinkedList<>();
......
......@@ -21,7 +21,7 @@ import com.google.common.collect.Lists;
import lombok.SneakyThrows;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.route.RouteUnit;
import org.apache.shardingsphere.core.route.SQLUnit;
import org.apache.shardingsphere.spi.database.DataSourceMetaData;
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute.sql.execute.result;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationDistinctProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationProjection;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import org.junit.Before;
import org.junit.Test;
import java.sql.SQLException;
import java.util.Collection;
import java.util.LinkedList;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public final class AggregationDistinctQueryMetaDataTest {
private AggregationDistinctQueryMetaData distinctQueryMetaData;
@Before
public void setUp() throws SQLException {
Collection<AggregationDistinctProjection> aggregationDistinctProjections = new LinkedList<>();
AggregationDistinctProjection distinctCountProjection = new AggregationDistinctProjection(0, 0, AggregationType.COUNT, "(DISTINCT order_id)", "c", "order_id");
AggregationDistinctProjection distinctAvgProjection = new AggregationDistinctProjection(0, 0, AggregationType.AVG, "(DISTINCT order_id)", "a", "order_id");
distinctAvgProjection.getDerivedAggregationProjections().add(new AggregationProjection(AggregationType.COUNT, "(DISTINCT order_id)", "AVG_DERIVED_COUNT_0"));
distinctAvgProjection.getDerivedAggregationProjections().add(new AggregationProjection(AggregationType.SUM, "(DISTINCT order_id)", "AVG_DERIVED_SUM_0"));
aggregationDistinctProjections.add(distinctCountProjection);
aggregationDistinctProjections.add(distinctAvgProjection);
Multimap<String, Integer> columnLabelAndIndexMap = HashMultimap.create();
columnLabelAndIndexMap.put("c", 1);
columnLabelAndIndexMap.put("a", 2);
columnLabelAndIndexMap.put("AVG_DERIVED_COUNT_0", 3);
columnLabelAndIndexMap.put("AVG_DERIVED_SUM_0", 4);
distinctQueryMetaData = new AggregationDistinctQueryMetaData(aggregationDistinctProjections, getQueryResultMetaData());
}
private QueryResultMetaData getQueryResultMetaData() throws SQLException {
QueryResultMetaData result = mock(QueryResultMetaData.class);
when(result.getColumnCount()).thenReturn(4);
when(result.getColumnLabel(1)).thenReturn("c");
when(result.getColumnLabel(2)).thenReturn("a");
when(result.getColumnLabel(3)).thenReturn("AVG_DERIVED_COUNT_0");
when(result.getColumnLabel(4)).thenReturn("AVG_DERIVED_SUM_0");
when(result.getColumnIndex("c")).thenReturn(1);
when(result.getColumnIndex("a")).thenReturn(2);
when(result.getColumnIndex("AVG_DERIVED_COUNT_0")).thenReturn(3);
when(result.getColumnIndex("AVG_DERIVED_SUM_0")).thenReturn(4);
return result;
}
@Test
public void assertIsAggregationDistinctColumnIndex() {
assertTrue(distinctQueryMetaData.isAggregationDistinctColumnIndex(1));
}
@Test
public void assertIsAggregationDistinctColumnLabel() {
assertTrue(distinctQueryMetaData.isAggregationDistinctColumnLabel("c"));
}
@Test
public void assertGetAggregationType() {
AggregationType actual = distinctQueryMetaData.getAggregationType(2);
assertThat(actual, is(AggregationType.AVG));
}
@Test
public void assertIsDerivedCountColumnIndex() {
assertTrue(distinctQueryMetaData.isDerivedCountColumnIndex(3));
}
@Test
public void assertIsDerivedSumColumnIndex() {
assertTrue(distinctQueryMetaData.isDerivedSumColumnIndex(4));
}
@Test
public void assertGetAggregationDistinctColumnIndexByColumnLabel() {
int actual = distinctQueryMetaData.getAggregationDistinctColumnIndex("a");
assertThat(actual, is(2));
}
@Test(expected = ShardingException.class)
public void assertGetAggregationDistinctColumnIndexByColumnLabelWithException() {
int actual = distinctQueryMetaData.getAggregationDistinctColumnIndex("f");
assertThat(actual, is(2));
}
@Test
public void assertGetAggregationDistinctColumnIndexBySumIndex() {
int actual = distinctQueryMetaData.getAggregationDistinctColumnIndex(4);
assertThat(actual, is(2));
}
@Test(expected = ShardingException.class)
public void assertGetAggregationDistinctColumnIndexBySumIndexWithException() {
int actual = distinctQueryMetaData.getAggregationDistinctColumnIndex(0);
assertThat(actual, is(2));
}
@Test
public void assertGetAggregationDistinctColumnLabel() {
String actual = distinctQueryMetaData.getAggregationDistinctColumnLabel(1);
assertThat(actual, is("c"));
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute.sql.execute.result;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationDistinctProjection;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import org.junit.Before;
import org.junit.Test;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.ObjectOutputStream;
import java.sql.SQLException;
import java.util.Calendar;
import java.util.Collection;
import java.util.LinkedList;
import java.util.List;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public final class AggregationDistinctQueryResultTest {
private AggregationDistinctQueryResult aggregationDistinctQueryResult;
@Before
public void setUp() throws SQLException {
aggregationDistinctQueryResult = new AggregationDistinctQueryResult(getQueryResults(), getAggregationDistinctProjections());
}
private Collection<QueryResult> getQueryResults() throws SQLException {
Collection<QueryResult> result = new LinkedList<>();
for (int i = 1; i <= 2; i++) {
QueryResult queryResult = mock(QueryResult.class);
when(queryResult.next()).thenReturn(true).thenReturn(false);
when(queryResult.getColumnCount()).thenReturn(5);
when(queryResult.getColumnLabel(1)).thenReturn("order_id");
when(queryResult.getColumnLabel(2)).thenReturn("c");
when(queryResult.getColumnLabel(3)).thenReturn("a");
when(queryResult.getColumnLabel(4)).thenReturn("AVG_DERIVED_COUNT_0");
when(queryResult.getColumnLabel(5)).thenReturn("AVG_DERIVED_SUM_0");
when(queryResult.getValue(1, Object.class)).thenReturn(10 * i);
when(queryResult.getValue(2, Object.class)).thenReturn(10 * i);
when(queryResult.getValue(3, Object.class)).thenReturn(10 * i);
when(queryResult.getValue(4, Object.class)).thenReturn(10 * i);
when(queryResult.getValue(5, Object.class)).thenReturn(10 * i);
doReturn(getQueryResultMetaData()).when(queryResult).getQueryResultMetaData();
result.add(queryResult);
result.add(queryResult);
}
return result;
}
private QueryResultMetaData getQueryResultMetaData() throws SQLException {
QueryResultMetaData result = mock(QueryResultMetaData.class);
when(result.getColumnCount()).thenReturn(5);
when(result.getColumnLabel(1)).thenReturn("order_id");
when(result.getColumnLabel(2)).thenReturn("c");
when(result.getColumnLabel(3)).thenReturn("a");
when(result.getColumnLabel(4)).thenReturn("AVG_DERIVED_COUNT_0");
when(result.getColumnLabel(5)).thenReturn("AVG_DERIVED_SUM_0");
when(result.getColumnIndex("order_id")).thenReturn(1);
when(result.getColumnIndex("c")).thenReturn(2);
when(result.getColumnIndex("a")).thenReturn(3);
when(result.getColumnIndex("AVG_DERIVED_COUNT_0")).thenReturn(4);
when(result.getColumnIndex("AVG_DERIVED_SUM_0")).thenReturn(5);
return result;
}
private List<AggregationDistinctProjection> getAggregationDistinctProjections() {
List<AggregationDistinctProjection> result = new LinkedList<>();
AggregationDistinctProjection distinctCountProjection = new AggregationDistinctProjection(0, 0, AggregationType.COUNT, "(DISTINCT order_id)", "c", "order_id");
AggregationDistinctProjection distinctAvgProjection = new AggregationDistinctProjection(0, 0, AggregationType.AVG, "(DISTINCT order_id)", "a", "order_id");
distinctAvgProjection.getDerivedAggregationProjections().add(new AggregationDistinctProjection(0, 0, AggregationType.COUNT, "(DISTINCT order_id)", "AVG_DERIVED_COUNT_0", "order_id"));
distinctAvgProjection.getDerivedAggregationProjections().add(new AggregationDistinctProjection(0, 0, AggregationType.SUM, "(DISTINCT order_id)", "AVG_DERIVED_SUM_0", "order_id"));
result.add(distinctCountProjection);
result.add(distinctAvgProjection);
return result;
}
@Test
public void assertDivide() throws SQLException {
List<DistinctQueryResult> actual = aggregationDistinctQueryResult.divide();
assertThat(actual.size(), is(2));
assertThat(actual.iterator().next().getColumnCount(), is((Object) 5));
}
@Test
public void assertGetValueByColumnIndex() {
aggregationDistinctQueryResult.next();
assertThat(aggregationDistinctQueryResult.getValue(1, Object.class), is((Object) 10));
assertThat(aggregationDistinctQueryResult.getValue(2, Object.class), is((Object) 1));
assertThat(aggregationDistinctQueryResult.getValue(3, Object.class), is((Object) 10));
assertThat(aggregationDistinctQueryResult.getValue(4, Object.class), is((Object) 1));
assertThat(aggregationDistinctQueryResult.getValue(5, Object.class), is((Object) 10));
}
@Test
public void assertGetValueByColumnLabel() {
aggregationDistinctQueryResult.next();
assertThat(aggregationDistinctQueryResult.getValue("order_id", Object.class), is((Object) 10));
assertThat(aggregationDistinctQueryResult.getValue("a", Object.class), is((Object) 10));
}
@Test
public void assertGetCalendarValueByColumnIndex() {
aggregationDistinctQueryResult.next();
assertThat(aggregationDistinctQueryResult.getCalendarValue(1, Object.class, Calendar.getInstance()), is((Object) 10));
}
@Test
public void assertGetCalendarValueByColumnLabel() {
aggregationDistinctQueryResult.next();
assertThat(aggregationDistinctQueryResult.getCalendarValue("order_id", Object.class, Calendar.getInstance()), is((Object) 10));
}
@Test
public void assertGetInputStreamByColumnIndex() throws IOException {
aggregationDistinctQueryResult.next();
assertThat(aggregationDistinctQueryResult.getInputStream(1, "Unicode").read(), is(getInputStream(10).read()));
}
@Test
public void assertGetInputStreamByColumnLabel() throws IOException {
aggregationDistinctQueryResult.next();
assertThat(aggregationDistinctQueryResult.getInputStream("order_id", "Unicode").read(), is(getInputStream(10).read()));
}
@Test
public void assertWasNull() {
assertTrue(aggregationDistinctQueryResult.wasNull());
}
private InputStream getInputStream(final Object value) throws IOException {
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
ObjectOutputStream objectOutputStream = new ObjectOutputStream(byteArrayOutputStream);
objectOutputStream.writeObject(value);
objectOutputStream.flush();
objectOutputStream.close();
return new ByteArrayInputStream(byteArrayOutputStream.toByteArray());
}
@Test
public void assertGetColumnCount() throws SQLException {
assertThat(aggregationDistinctQueryResult.getColumnCount(), is(5));
}
@Test
public void assertGetColumnLabel() throws SQLException {
assertThat(aggregationDistinctQueryResult.getColumnLabel(3), is("a"));
assertThat(aggregationDistinctQueryResult.getColumnLabel(1), is("order_id"));
}
@Test(expected = SQLException.class)
public void assertGetColumnLabelWithException() throws SQLException {
assertThat(aggregationDistinctQueryResult.getColumnLabel(6), is("order_id"));
}
@Test
public void assertGetColumnIndex() {
assertThat(aggregationDistinctQueryResult.getColumnIndex("c"), is(2));
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute.sql.execute.result;
import org.junit.Before;
import org.junit.Test;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.ObjectOutputStream;
import java.sql.SQLException;
import java.util.Calendar;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public final class DistinctQueryResultTest {
private DistinctQueryResult distinctQueryResult;
private QueryResultMetaData queryResultMetaData;
@Before
public void setUp() throws SQLException {
queryResultMetaData = getQueryResultMetaData();
Collection<QueryResult> queryResults = getQueryResults();
List<String> distinctColumnLabels = Collections.singletonList("order_id");
distinctQueryResult = new DistinctQueryResult(queryResults, distinctColumnLabels);
}
private Collection<QueryResult> getQueryResults() throws SQLException {
Collection<QueryResult> result = new LinkedList<>();
for (int i = 1; i <= 2; i++) {
QueryResult queryResult = mock(QueryResult.class);
when(queryResult.next()).thenReturn(true).thenReturn(false);
when(queryResult.getColumnCount()).thenReturn(1);
when(queryResult.getColumnLabel(1)).thenReturn("order_id");
when(queryResult.getValue(1, Object.class)).thenReturn(10 * i);
when(queryResult.isCaseSensitive(1)).thenReturn(true);
doReturn(queryResultMetaData).when(queryResult).getQueryResultMetaData();
result.add(queryResult);
result.add(queryResult);
}
return result;
}
private QueryResultMetaData getQueryResultMetaData() throws SQLException {
QueryResultMetaData result = mock(QueryResultMetaData.class);
when(result.getColumnCount()).thenReturn(1);
when(result.getColumnLabel(1)).thenReturn("order_id");
when(result.getColumnIndex("order_id")).thenReturn(1);
return result;
}
@Test
public void assertDivide() throws SQLException {
List<DistinctQueryResult> actual = distinctQueryResult.divide();
assertThat(actual.size(), is(2));
assertThat(actual.iterator().next().getColumnCount(), is((Object) 1));
}
@Test
public void assertNext() {
assertTrue(distinctQueryResult.next());
assertTrue(distinctQueryResult.next());
assertFalse(distinctQueryResult.next());
}
@Test
public void assertGetValueByColumnIndex() {
distinctQueryResult.next();
assertThat(distinctQueryResult.getValue(1, Object.class), is((Object) 10));
}
@Test
public void assertGetValueByColumnLabel() {
distinctQueryResult.next();
assertThat(distinctQueryResult.getValue("order_id", Object.class), is((Object) 10));
}
@Test
public void assertGetCalendarValueByColumnIndex() {
distinctQueryResult.next();
assertThat(distinctQueryResult.getCalendarValue(1, Object.class, Calendar.getInstance()), is((Object) 10));
}
@Test
public void assertGetCalendarValueByColumnLabel() {
distinctQueryResult.next();
assertThat(distinctQueryResult.getCalendarValue("order_id", Object.class, Calendar.getInstance()), is((Object) 10));
}
@Test
public void assertGetInputStreamByColumnIndex() throws IOException {
distinctQueryResult.next();
assertThat(distinctQueryResult.getInputStream(1, "Unicode").read(), is(getInputStream(10).read()));
}
private InputStream getInputStream(final Object value) throws IOException {
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
ObjectOutputStream objectOutputStream = new ObjectOutputStream(byteArrayOutputStream);
objectOutputStream.writeObject(value);
objectOutputStream.flush();
objectOutputStream.close();
return new ByteArrayInputStream(byteArrayOutputStream.toByteArray());
}
@Test
public void assertGetInputStreamByColumnLabel() throws IOException {
distinctQueryResult.next();
assertThat(distinctQueryResult.getInputStream("order_id", "Unicode").read(), is(getInputStream(10).read()));
}
@Test
public void assertWasNull() {
assertTrue(distinctQueryResult.wasNull());
}
@Test
public void assertIsCaseSensitive() throws SQLException {
when(queryResultMetaData.isCaseSensitive(1)).thenReturn(true);
assertTrue(distinctQueryResult.isCaseSensitive(1));
}
@Test
public void assertGetColumnCount() throws SQLException {
assertThat(distinctQueryResult.getColumnCount(), is(1));
}
@Test
public void assertGetColumnLabel() throws SQLException {
assertThat(distinctQueryResult.getColumnLabel(1), is("order_id"));
}
@Test(expected = SQLException.class)
public void assertGetColumnLabelWithException() throws SQLException {
assertThat(distinctQueryResult.getColumnLabel(2), is("order_id"));
}
@Test
public void assertGetColumnIndex() {
assertThat(distinctQueryResult.getColumnIndex("order_id"), is(1));
}
@Test
public void assertGetQueryResultMetaData() {
assertThat(distinctQueryResult.getQueryResultMetaData(), is(queryResultMetaData));
}
@Test
public void assertGetResultData() {
assertThat(distinctQueryResult.getResultData().next().getValue(1), is((Object) 10));
}
@Test
public void assertGetCurrentRow() {
distinctQueryResult.next();
assertThat(distinctQueryResult.getCurrentRow().getValue(1), is((Object) 10));
}
@Test(expected = SQLException.class)
public void assertGetColumnLabelAndIndexMapWithException() throws SQLException {
QueryResult queryResult = mock(QueryResult.class);
when(queryResult.next()).thenReturn(true).thenReturn(false);
when(queryResult.getColumnCount()).thenThrow(SQLException.class);
when(queryResult.getColumnLabel(1)).thenReturn("order_id");
when(queryResult.getValue(1, Object.class)).thenReturn(10);
Collection<QueryResult> queryResults = new LinkedList<>();
queryResults.add(queryResult);
List<String> distinctColumnLabels = Collections.singletonList("order_id");
distinctQueryResult = new DistinctQueryResult(queryResults, distinctColumnLabels);
}
@Test(expected = SQLException.class)
public void assertGetResultDataWithException() throws SQLException {
QueryResult queryResult = mock(QueryResult.class);
when(queryResult.next()).thenThrow(SQLException.class);
when(queryResult.getColumnCount()).thenReturn(1);
when(queryResult.getColumnLabel(1)).thenReturn("order_id");
when(queryResult.getValue(1, Object.class)).thenReturn(10);
Collection<QueryResult> queryResults = new LinkedList<>();
queryResults.add(queryResult);
List<String> distinctColumnLabels = Collections.singletonList("order_id");
distinctQueryResult = new DistinctQueryResult(queryResults, distinctColumnLabels);
}
}
......@@ -57,18 +57,6 @@ public final class MemoryQueryResultTest {
private final ShardingEncryptor shardingEncryptor = mock(ShardingEncryptor.class);
@Test
public void assertConstructorWithShardingRule() throws SQLException {
MemoryQueryResult queryResult = new MemoryQueryResult(getResultSet(), getShardingRule(), new ShardingProperties(new Properties()), getSqlStatementContext());
assertThat(queryResult.getQueryResultMetaData().getShardingEncryptor(1), is(Optional.fromNullable(shardingEncryptor)));
}
@Test
public void assertConstructorWithEncryptRule() throws SQLException {
MemoryQueryResult queryResult = new MemoryQueryResult(getResultSet(), getEncryptRule(), new ShardingProperties(new Properties()), getSqlStatementContext());
assertThat(queryResult.getQueryResultMetaData().getShardingEncryptor(1), is(Optional.fromNullable(shardingEncryptor)));
}
@Test(expected = SQLException.class)
@SneakyThrows
public void assertConstructorWithSqlException() {
......
......@@ -19,12 +19,12 @@ package org.apache.shardingsphere.core.execute.sql.execute.result;
import com.google.common.base.Optional;
import org.apache.shardingsphere.core.constant.properties.ShardingProperties;
import org.apache.shardingsphere.sql.parser.relation.segment.table.TablesContext;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.core.rule.EncryptRule;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.core.strategy.encrypt.EncryptTable;
import org.apache.shardingsphere.spi.encrypt.ShardingEncryptor;
import org.apache.shardingsphere.sql.parser.relation.segment.table.TablesContext;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.hamcrest.core.Is;
import org.junit.Before;
import org.junit.Test;
......@@ -41,7 +41,6 @@ import java.util.Calendar;
import java.util.Collections;
import java.util.Properties;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
......@@ -65,18 +64,6 @@ public final class StreamQueryResultTest {
when(sqlStatementContext.getTablesContext()).thenReturn(tablesContext);
when(tablesContext.getTableNames()).thenReturn(Collections.singleton("order"));
}
@Test
public void assertConstructorWithShardingRule() throws SQLException {
StreamQueryResult queryResult = new StreamQueryResult(getResultSet(), getShardingRule(), new ShardingProperties(new Properties()), sqlStatementContext);
assertThat(queryResult.getQueryResultMetaData().getShardingEncryptor(1), is(Optional.fromNullable(shardingEncryptor)));
}
@Test
public void assertConstructorWithEncryptRule() throws SQLException {
StreamQueryResult queryResult = new StreamQueryResult(getResultSet(), getEncryptRule(), new ShardingProperties(new Properties()), sqlStatementContext);
assertThat(queryResult.getQueryResultMetaData().getShardingEncryptor(1), is(Optional.fromNullable(shardingEncryptor)));
}
private ShardingRule getShardingRule() {
ShardingRule result = mock(ShardingRule.class);
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.execute.sql.execute.row;
import org.junit.Before;
import org.junit.Test;
import java.util.Collections;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
public final class QueryRowTest {
private QueryRow queryRow;
@Before
public void setUp() {
queryRow = new QueryRow(Collections.singletonList((Object) 10), Collections.singletonList(1));
}
@Test
public void assertGetValue() {
assertThat(queryRow.getValue(1), is((Object) 10));
}
@Test
public void assertEqual() {
QueryRow queryRow1 = new QueryRow(Collections.singletonList((Object) 10));
assertTrue(queryRow1.equals(queryRow));
}
@Test
public void assertEqualPartly() {
QueryRow queryRow1 = new QueryRow(Collections.singletonList((Object) 10), Collections.singletonList(1));
QueryRow queryRow2 = new QueryRow(Collections.singletonList((Object) 8), Collections.singletonList(1));
assertTrue(queryRow.equals(queryRow1));
assertFalse(queryRow.equals(queryRow2));
}
@Test
public void assertEqualPartlyWithNull() {
QueryRow queryRowWithNull = new QueryRow(Collections.singletonList(null), Collections.singletonList(1));
QueryRow queryRowWithNull2 = new QueryRow(Collections.singletonList(null), Collections.singletonList(1));
QueryRow queryRowWithNonNull = new QueryRow(Collections.singletonList((Object) 8), Collections.singletonList(1));
assertTrue(queryRowWithNull.equals(queryRowWithNull2));
assertFalse(queryRowWithNull.equals(queryRowWithNonNull));
}
@Test
public void assertHashCode() {
assertThat(queryRow.hashCode(), is(41));
}
@Test
public void assertGetRowData() {
assertThat(queryRow.getRowData(), is(Collections.singletonList((Object) 10)));
}
@Test
public void assertGetDistinctColumnIndexes() {
assertThat(queryRow.getDistinctColumnIndexes(), is(Collections.singletonList(1)));
}
}
......@@ -18,8 +18,8 @@
package org.apache.shardingsphere.core.execute.sql.prepare;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.route.RouteUnit;
import org.apache.shardingsphere.core.route.SQLUnit;
import org.junit.Test;
......
......@@ -55,7 +55,7 @@ public final class MergeEngineFactory {
public static MergeEngine newInstance(final DatabaseType databaseType, final ShardingRule shardingRule,
final SQLRouteResult routeResult, final TableMetas tableMetas, final List<QueryResult> queryResults) throws SQLException {
if (routeResult.getSqlStatementContext() instanceof SelectSQLStatementContext) {
return new DQLMergeEngine(databaseType, tableMetas, routeResult, queryResults);
return new DQLMergeEngine(databaseType, (SelectSQLStatementContext) routeResult.getSqlStatementContext(), queryResults);
}
if (routeResult.getSqlStatementContext().getSqlStatement() instanceof DALStatement) {
return new DALMergeEngine(shardingRule, queryResults, routeResult.getSqlStatementContext(), tableMetas);
......
......@@ -17,12 +17,8 @@
package org.apache.shardingsphere.core.merge.dql;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
import lombok.Getter;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.sql.execute.result.AggregationDistinctQueryResult;
import org.apache.shardingsphere.core.execute.sql.execute.result.DistinctQueryResult;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.MergeEngine;
import org.apache.shardingsphere.core.merge.MergedResult;
......@@ -33,19 +29,15 @@ import org.apache.shardingsphere.core.merge.dql.orderby.OrderByStreamMergedResul
import org.apache.shardingsphere.core.merge.dql.pagination.LimitDecoratorMergedResult;
import org.apache.shardingsphere.core.merge.dql.pagination.RowNumberDecoratorMergedResult;
import org.apache.shardingsphere.core.merge.dql.pagination.TopAndRowNumberDecoratorMergedResult;
import org.apache.shardingsphere.core.metadata.table.TableMetaData;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.sql.parser.relation.metadata.RelationMetaData;
import org.apache.shardingsphere.sql.parser.relation.metadata.RelationMetas;
import org.apache.shardingsphere.spi.database.DatabaseType;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
import org.apache.shardingsphere.sql.parser.relation.segment.select.pagination.PaginationContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationDistinctProjection;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
import org.apache.shardingsphere.core.route.SQLRouteResult;
import org.apache.shardingsphere.spi.database.DatabaseType;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.IndexOrderByItemSegment;
import org.apache.shardingsphere.sql.parser.util.SQLUtil;
import java.sql.SQLException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
......@@ -56,63 +48,23 @@ import java.util.TreeMap;
* @author zhangliang
* @author panjuan
*/
@RequiredArgsConstructor
public final class DQLMergeEngine implements MergeEngine {
private final DatabaseType databaseType;
private final SQLRouteResult routeResult;
private final SelectSQLStatementContext selectSQLStatementContext;
private final List<QueryResult> queryResults;
@Getter
private final Map<String, Integer> columnLabelIndexMap;
public DQLMergeEngine(final DatabaseType databaseType, final TableMetas tableMetas, final SQLRouteResult routeResult, final List<QueryResult> queryResults) throws SQLException {
this.databaseType = databaseType;
this.routeResult = routeResult;
this.selectSQLStatementContext = (SelectSQLStatementContext) routeResult.getSqlStatementContext();
this.queryResults = getRealQueryResults(getRelationMetas(tableMetas), queryResults);
columnLabelIndexMap = getColumnLabelIndexMap(this.queryResults.get(0));
}
private RelationMetas getRelationMetas(final TableMetas tableMetas) {
Map<String, RelationMetaData> result = new HashMap<>(tableMetas.getAllTableNames().size());
for (String each : tableMetas.getAllTableNames()) {
TableMetaData tableMetaData = tableMetas.get(each);
result.put(each, new RelationMetaData(tableMetaData.getColumns().keySet()));
}
return new RelationMetas(result);
}
private List<QueryResult> getRealQueryResults(final RelationMetas relationMetas, final List<QueryResult> queryResults) throws SQLException {
List<QueryResult> result = queryResults;
if (1 == result.size()) {
return result;
}
List<AggregationDistinctProjection> aggregationDistinctProjections = selectSQLStatementContext.getProjectionsContext().getAggregationDistinctProjections();
if (!aggregationDistinctProjections.isEmpty()) {
result = getDividedQueryResults(new AggregationDistinctQueryResult(queryResults, aggregationDistinctProjections));
}
if (isDistinctRowSelectItems()) {
result = getDividedQueryResults(new DistinctQueryResult(queryResults, selectSQLStatementContext.getColumnLabels(relationMetas)));
@Override
public MergedResult merge() throws SQLException {
if (1 == queryResults.size()) {
return new IteratorStreamMergedResult(queryResults);
}
return result.isEmpty() ? queryResults : result;
}
private List<QueryResult> getDividedQueryResults(final DistinctQueryResult distinctQueryResult) {
return Lists.transform(distinctQueryResult.divide(), new Function<DistinctQueryResult, QueryResult>() {
@Override
public QueryResult apply(final DistinctQueryResult input) {
return input;
}
});
}
private boolean isDistinctRowSelectItems() {
return selectSQLStatementContext.getProjectionsContext().isDistinctRow() && selectSQLStatementContext.getGroupByContext().getItems().isEmpty();
Map<String, Integer> columnLabelIndexMap = getColumnLabelIndexMap(queryResults.get(0));
selectSQLStatementContext.setIndexes(columnLabelIndexMap);
return decorate(build(columnLabelIndexMap));
}
private Map<String, Integer> getColumnLabelIndexMap(final QueryResult queryResult) throws SQLException {
......@@ -123,33 +75,48 @@ public final class DQLMergeEngine implements MergeEngine {
return result;
}
@Override
public MergedResult merge() throws SQLException {
if (1 == queryResults.size()) {
return new IteratorStreamMergedResult(queryResults);
private MergedResult build(final Map<String, Integer> columnLabelIndexMap) throws SQLException {
if (isNeedProcessGroupBy()) {
return getGroupByMergedResult(columnLabelIndexMap);
}
selectSQLStatementContext.setIndexes(columnLabelIndexMap);
return decorate(build());
}
private MergedResult build() throws SQLException {
if (!selectSQLStatementContext.getGroupByContext().getItems().isEmpty() || !selectSQLStatementContext.getProjectionsContext().getAggregationProjections().isEmpty()) {
return getGroupByMergedResult();
if (isNeedProcessDistinctRow()) {
setGroupByForDistinctRow();
return getGroupByMergedResult(columnLabelIndexMap);
}
if (!selectSQLStatementContext.getOrderByContext().getItems().isEmpty()) {
if (isNeedProcessOrderBy()) {
return new OrderByStreamMergedResult(queryResults, selectSQLStatementContext.getOrderByContext().getItems());
}
return new IteratorStreamMergedResult(queryResults);
}
private MergedResult getGroupByMergedResult() throws SQLException {
private boolean isNeedProcessGroupBy() {
return !selectSQLStatementContext.getGroupByContext().getItems().isEmpty() || !selectSQLStatementContext.getProjectionsContext().getAggregationProjections().isEmpty();
}
private boolean isNeedProcessDistinctRow() {
return selectSQLStatementContext.getProjectionsContext().isDistinctRow();
}
private void setGroupByForDistinctRow() {
for (int index = 1; index <= selectSQLStatementContext.getProjectionsContext().getColumnLabels().size(); index++) {
OrderByItem orderByItem = new OrderByItem(new IndexOrderByItemSegment(-1, -1, index, OrderDirection.ASC, OrderDirection.ASC));
orderByItem.setIndex(index);
selectSQLStatementContext.getGroupByContext().getItems().add(orderByItem);
}
}
private MergedResult getGroupByMergedResult(final Map<String, Integer> columnLabelIndexMap) throws SQLException {
return selectSQLStatementContext.isSameGroupByAndOrderByItems()
? new GroupByStreamMergedResult(columnLabelIndexMap, queryResults, selectSQLStatementContext)
: new GroupByMemoryMergedResult(columnLabelIndexMap, queryResults, selectSQLStatementContext);
}
private boolean isNeedProcessOrderBy() {
return !selectSQLStatementContext.getOrderByContext().getItems().isEmpty();
}
private MergedResult decorate(final MergedResult mergedResult) throws SQLException {
PaginationContext paginationContext = ((SelectSQLStatementContext) routeResult.getSqlStatementContext()).getPaginationContext();
PaginationContext paginationContext = selectSQLStatementContext.getPaginationContext();
if (!paginationContext.isHasPagination() || 1 == queryResults.size()) {
return mergedResult;
}
......
......@@ -26,6 +26,7 @@ import org.apache.shardingsphere.core.merge.dql.common.MemoryMergedResult;
import org.apache.shardingsphere.core.merge.dql.common.MemoryQueryResultRow;
import org.apache.shardingsphere.core.merge.dql.groupby.aggregation.AggregationUnit;
import org.apache.shardingsphere.core.merge.dql.groupby.aggregation.AggregationUnitFactory;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationDistinctProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationProjection;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
......@@ -87,7 +88,7 @@ public final class GroupByMemoryMergedResult extends MemoryMergedResult {
@Override
public AggregationUnit apply(final AggregationProjection input) {
return AggregationUnitFactory.create(input.getType());
return AggregationUnitFactory.create(input.getType(), input instanceof AggregationDistinctProjection);
}
});
aggregationMap.put(groupByValue, map);
......
......@@ -24,6 +24,7 @@ import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.dql.groupby.aggregation.AggregationUnit;
import org.apache.shardingsphere.core.merge.dql.groupby.aggregation.AggregationUnitFactory;
import org.apache.shardingsphere.core.merge.dql.orderby.OrderByStreamMergedResult;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationDistinctProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationProjection;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
......@@ -82,7 +83,7 @@ public final class GroupByStreamMergedResult extends OrderByStreamMergedResult {
@Override
public AggregationUnit apply(final AggregationProjection input) {
return AggregationUnitFactory.create(input.getType());
return AggregationUnitFactory.create(input.getType(), input instanceof AggregationDistinctProjection);
}
});
while (currentGroupByValues.equals(new GroupByValue(getCurrentQueryResult(), selectSQLStatementContext.getGroupByContext().getItems()).getGroupValues())) {
......
......@@ -33,19 +33,21 @@ public final class AggregationUnitFactory {
* Create aggregation unit instance.
*
* @param type aggregation function type
* @param isDistinct is distinct
* @return aggregation unit instance
*/
public static AggregationUnit create(final AggregationType type) {
public static AggregationUnit create(final AggregationType type, final boolean isDistinct) {
switch (type) {
case MAX:
return new ComparableAggregationUnit(false);
case MIN:
return new ComparableAggregationUnit(true);
case SUM:
return isDistinct ? new DistinctSumAggregationUnit() : new AccumulationAggregationUnit();
case COUNT:
return new AccumulationAggregationUnit();
return isDistinct ? new DistinctCountAggregationUnit() : new AccumulationAggregationUnit();
case AVG:
return new AverageAggregationUnit();
return isDistinct ? new DistinctAverageAggregationUnit() : new AverageAggregationUnit();
default:
throw new UnsupportedOperationException(type.name());
}
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.merge.dql.groupby.aggregation;
import lombok.RequiredArgsConstructor;
import java.math.BigDecimal;
import java.util.Collection;
import java.util.LinkedHashSet;
import java.util.List;
/**
* Distinct average aggregation unit.
*
* @author zhangliang
*/
@RequiredArgsConstructor
public final class DistinctAverageAggregationUnit implements AggregationUnit {
private BigDecimal count;
private BigDecimal sum;
private Collection<Comparable<?>> countValues = new LinkedHashSet<>();
private Collection<Comparable<?>> sumValues = new LinkedHashSet<>();
@Override
public void merge(final List<Comparable<?>> values) {
if (null == values || null == values.get(0) || null == values.get(1)) {
return;
}
if (this.countValues.add(values.get(0)) && this.sumValues.add(values.get(0))) {
if (null == count) {
count = new BigDecimal("0");
}
if (null == sum) {
sum = new BigDecimal("0");
}
count = count.add(new BigDecimal(values.get(0).toString()));
sum = sum.add(new BigDecimal(values.get(1).toString()));
}
}
@Override
public Comparable<?> getResult() {
if (null == count || BigDecimal.ZERO.equals(count)) {
return count;
}
// TODO use metadata to fetch float number precise for database field
return sum.divide(count, 4, BigDecimal.ROUND_HALF_UP);
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.merge.dql.groupby.aggregation;
import lombok.RequiredArgsConstructor;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
/**
* Distinct count aggregation unit.
*
* @author zhangliang
*/
@RequiredArgsConstructor
public final class DistinctCountAggregationUnit implements AggregationUnit {
private Collection<Comparable<?>> values = new HashSet<>();
@Override
public void merge(final List<Comparable<?>> values) {
if (null == values || null == values.get(0)) {
return;
}
this.values.add(values.get(0));
}
@Override
public Comparable<?> getResult() {
return values.size();
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.core.merge.dql.groupby.aggregation;
import lombok.RequiredArgsConstructor;
import java.math.BigDecimal;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
/**
* Distinct sum aggregation unit.
*
* @author zhangliang
*/
@RequiredArgsConstructor
public final class DistinctSumAggregationUnit implements AggregationUnit {
private BigDecimal result;
private Collection<Comparable<?>> values = new HashSet<>();
@Override
public void merge(final List<Comparable<?>> values) {
if (null == values || null == values.get(0)) {
return;
}
if (this.values.add(values.get(0))) {
if (null == result) {
result = new BigDecimal("0");
}
result = result.add(new BigDecimal(values.get(0).toString()));
}
}
@Override
public Comparable<?> getResult() {
return result;
}
}
......@@ -22,7 +22,7 @@ import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.dal.DALMergeEngine;
import org.apache.shardingsphere.core.merge.dql.DQLMergeEngine;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext;
......@@ -69,7 +69,7 @@ public final class MergeEngineFactoryTest {
when(resultSetMetaData.getColumnLabel(1)).thenReturn("label");
List<ResultSet> resultSets = Lists.newArrayList(resultSet);
queryResults = new ArrayList<>(resultSets.size());
queryResults.add(new TestQueryResult(resultSets.get(0)));
queryResults.add(new ResultSetBasedQueryResultFixture(resultSets.get(0)));
}
@Test
......@@ -77,7 +77,7 @@ public final class MergeEngineFactoryTest {
SQLRouteResult routeResult = new SQLRouteResult(
new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList())),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList())),
new ShardingConditions(Collections.<ShardingCondition>emptyList()));
assertThat(MergeEngineFactory.newInstance(DatabaseTypes.getActualDatabaseType("MySQL"), null, routeResult, mock(TableMetas.class), queryResults), instanceOf(DQLMergeEngine.class));
}
......
......@@ -23,7 +23,7 @@ import org.apache.shardingsphere.core.merge.dal.show.ShowCreateTableMergedResult
import org.apache.shardingsphere.core.merge.dal.show.ShowDatabasesMergedResult;
import org.apache.shardingsphere.core.merge.dal.show.ShowOtherMergedResult;
import org.apache.shardingsphere.core.merge.dal.show.ShowTablesMergedResult;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.sql.parser.sql.statement.dal.DALStatement;
import org.apache.shardingsphere.sql.parser.sql.statement.dal.dialect.mysql.DescribeStatement;
......@@ -51,7 +51,7 @@ public final class DALMergeEngineTest {
@Before
public void setUp() {
ResultSet resultSet = mock(ResultSet.class);
queryResults = Collections.<QueryResult>singletonList(new TestQueryResult(resultSet));
queryResults = Collections.<QueryResult>singletonList(new ResultSetBasedQueryResultFixture(resultSet));
}
@Test
......
......@@ -20,12 +20,12 @@ package org.apache.shardingsphere.core.merge.dal.desc;
import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.fixture.DescribeQueryResultFixture;
import org.apache.shardingsphere.sql.parser.relation.segment.table.TablesContext;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.core.merge.fixture.QueryResultBasedQueryResultFixture;
import org.apache.shardingsphere.core.rule.EncryptRule;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.core.strategy.encrypt.EncryptTable;
import org.apache.shardingsphere.sql.parser.relation.segment.table.TablesContext;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.junit.Before;
import org.junit.Test;
......@@ -42,11 +42,11 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public final class DescribeTableMergedResultTest {
private List<QueryResult> queryResults;
private SQLStatementContext sqlStatementContext;
@Before
public void setUp() throws SQLException {
sqlStatementContext = mock(SQLStatementContext.class);
......@@ -54,10 +54,10 @@ public final class DescribeTableMergedResultTest {
when(tablesContext.getSingleTableName()).thenReturn("user");
when(sqlStatementContext.getTablesContext()).thenReturn(tablesContext);
List<QueryResult> mockQueryResults = mockQueryResults();
QueryResult queryResult = new DescribeQueryResultFixture(mockQueryResults.iterator());
QueryResult queryResult = new QueryResultBasedQueryResultFixture(mockQueryResults.iterator());
queryResults = Lists.newArrayList(queryResult);
}
private List<QueryResult> mockQueryResults() throws SQLException {
List<QueryResult> queryResults = new LinkedList<>();
queryResults.add(mockQueryResult(6, "id", "int(11) unsigned", "NO", "PRI", "", "auto_increment"));
......@@ -66,7 +66,7 @@ public final class DescribeTableMergedResultTest {
queryResults.add(mockQueryResult(6, "name_assisted", "varchar(100)", "YES", "", "", ""));
return queryResults;
}
private QueryResult mockQueryResult(final int columnNum, final String field, final String type, final String nullValue, final String key, final String defaultValue,
final String extra) throws SQLException {
QueryResult queryResult = mock(QueryResult.class);
......@@ -97,7 +97,7 @@ public final class DescribeTableMergedResultTest {
when(queryResult.getValue("Extra", Object.class)).thenReturn(extra);
return queryResult;
}
@Test
public void assertNextForEmptyQueryResult() throws SQLException {
ShardingRule shardingRule = mock(ShardingRule.class);
......@@ -105,7 +105,7 @@ public final class DescribeTableMergedResultTest {
DescribeTableMergedResult describeTableMergedResult = new DescribeTableMergedResult(shardingRule, queryResults, sqlStatementContext);
assertFalse(describeTableMergedResult.next());
}
@Test
public void assertFieldWithEncryptRule() throws SQLException {
ShardingRule shardingRule = mockShardingRuleWithEncryptRule();
......@@ -121,7 +121,7 @@ public final class DescribeTableMergedResultTest {
assertThat(describeTableMergedResult.getValue("Field", String.class).toString(), is("pre_name"));
assertFalse(describeTableMergedResult.next());
}
@Test
public void assertFieldWithoutEncryptRule() throws SQLException {
ShardingRule shardingRule = mockShardingRuleWithoutEncryptRule();
......@@ -139,7 +139,7 @@ public final class DescribeTableMergedResultTest {
assertThat(describeTableMergedResult.getValue(1, String.class).toString(), is("name_assisted"));
assertThat(describeTableMergedResult.getValue("Field", String.class).toString(), is("name_assisted"));
}
@Test
public void assertAllWithoutEncryptRule() throws SQLException {
ShardingRule shardingRule = mockShardingRuleWithoutEncryptRule();
......@@ -158,7 +158,7 @@ public final class DescribeTableMergedResultTest {
assertThat(describeTableMergedResult.getValue(6, String.class).toString(), is("auto_increment"));
assertThat(describeTableMergedResult.getValue("Extra", String.class).toString(), is("auto_increment"));
}
@Test
public void assertAllWithEncryptRule() throws SQLException {
ShardingRule shardingRule = mockShardingRuleWithEncryptRule();
......@@ -177,7 +177,7 @@ public final class DescribeTableMergedResultTest {
assertThat(describeTableMergedResult.getValue(6, String.class).toString(), is("auto_increment"));
assertThat(describeTableMergedResult.getValue("Extra", String.class).toString(), is("auto_increment"));
}
private ShardingRule mockShardingRuleWithoutEncryptRule() {
ShardingRule shardingRule = mock(ShardingRule.class);
EncryptRule encryptRule = mock(EncryptRule.class);
......@@ -185,7 +185,7 @@ public final class DescribeTableMergedResultTest {
when(shardingRule.getEncryptRule()).thenReturn(encryptRule);
return shardingRule;
}
private ShardingRule mockShardingRuleWithEncryptRule() {
ShardingRule shardingRule = mock(ShardingRule.class);
EncryptRule encryptRule = mock(EncryptRule.class);
......@@ -197,5 +197,4 @@ public final class DescribeTableMergedResultTest {
when(encryptTable.getLogicColumn("name")).thenReturn("logic_name");
return shardingRule;
}
}
......@@ -23,7 +23,7 @@ import org.apache.shardingsphere.api.config.sharding.TableRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.strategy.ComplexShardingStrategyConfiguration;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.fixture.ComplexKeysShardingAlgorithmFixture;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.core.metadata.column.ColumnMetaData;
import org.apache.shardingsphere.core.metadata.table.TableMetaData;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
......@@ -75,7 +75,7 @@ public final class ShowCreateTableMergedResultTest {
}
queryResults = new ArrayList<>(resultSets.size());
for (ResultSet each : resultSets) {
queryResults.add(new TestQueryResult(each));
queryResults.add(new ResultSetBasedQueryResultFixture(each));
}
}
......
......@@ -19,7 +19,7 @@ package org.apache.shardingsphere.core.merge.dal.show;
import org.apache.shardingsphere.core.constant.ShardingConstant;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.junit.Before;
import org.junit.Test;
......@@ -136,8 +136,8 @@ public final class ShowDatabasesMergedResultTest {
when(resultSetMetaData2.getColumnLabel(1)).thenReturn("SCHEMA_NAME");
List<QueryResult> queryResults = new ArrayList<>();
queryResults.add(new TestQueryResult(resultSet));
queryResults.add(new TestQueryResult(resultSet2));
queryResults.add(new ResultSetBasedQueryResultFixture(resultSet));
queryResults.add(new ResultSetBasedQueryResultFixture(resultSet2));
return new ShowDatabasesMergedResult(null, queryResults);
}
}
......@@ -17,7 +17,7 @@
package org.apache.shardingsphere.core.merge.dal.show;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
......@@ -58,7 +58,7 @@ public final class ShowOtherMergedResultTest {
@Before
public void setUp() {
showOtherMergedResult = new ShowOtherMergedResult(new TestQueryResult(resultSet));
showOtherMergedResult = new ShowOtherMergedResult(new ResultSetBasedQueryResultFixture(resultSet));
}
@Test
......
......@@ -23,7 +23,7 @@ import org.apache.shardingsphere.api.config.sharding.TableRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.strategy.ComplexShardingStrategyConfiguration;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.fixture.ComplexKeysShardingAlgorithmFixture;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.core.metadata.column.ColumnMetaData;
import org.apache.shardingsphere.core.metadata.table.TableMetaData;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
......@@ -75,7 +75,7 @@ public final class ShowTablesMergedResultTest {
}
queryResults = new ArrayList<>(resultSets.size());
for (ResultSet each : resultSets) {
queryResults.add(new TestQueryResult(each));
queryResults.add(new ResultSetBasedQueryResultFixture(each));
}
}
......
......@@ -17,7 +17,7 @@
package org.apache.shardingsphere.core.merge.dql.common;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
......@@ -47,7 +47,7 @@ public final class MemoryQueryResultRowTest {
when(resultSet.getMetaData()).thenReturn(resultSetMetaData);
when(resultSetMetaData.getColumnCount()).thenReturn(1);
when(resultSet.getObject(1)).thenReturn("value");
memoryResultSetRow = new MemoryQueryResultRow(new TestQueryResult(resultSet));
memoryResultSetRow = new MemoryQueryResultRow(new ResultSetBasedQueryResultFixture(resultSet));
}
@Test
......
......@@ -18,7 +18,7 @@
package org.apache.shardingsphere.core.merge.dql.common;
import org.apache.shardingsphere.core.merge.dql.common.fixture.TestStreamMergedResult;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
......@@ -59,7 +59,7 @@ public final class StreamMergedResultTest {
@Before
public void setUp() {
streamMergedResult = new TestStreamMergedResult();
streamMergedResult.setCurrentQueryResult(new TestQueryResult(resultSet));
streamMergedResult.setCurrentQueryResult(new ResultSetBasedQueryResultFixture(resultSet));
}
@Test(expected = SQLException.class)
......
......@@ -22,8 +22,9 @@ import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.MergedResult;
import org.apache.shardingsphere.core.merge.dql.DQLMergeEngine;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
......@@ -31,13 +32,7 @@ import org.apache.shardingsphere.sql.parser.relation.segment.select.pagination.P
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.Projection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.ProjectionsContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationProjection;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
import org.apache.shardingsphere.core.route.SQLRouteResult;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.IndexOrderByItemSegment;
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
import org.junit.Before;
......@@ -67,14 +62,14 @@ public final class GroupByMemoryMergedResultTest {
private List<QueryResult> queryResults;
private SQLRouteResult routeResult;
private SelectSQLStatementContext selectSQLStatementContext;
@Before
public void setUp() throws SQLException {
resultSets = Lists.newArrayList(mockResultSet(), mockResultSet(), mockResultSet());
queryResults = new ArrayList<>(resultSets.size());
for (ResultSet each : resultSets) {
queryResults.add(new TestQueryResult(each));
queryResults.add(new ResultSetBasedQueryResultFixture(each));
}
AggregationProjection aggregationSelectItem1 = new AggregationProjection(AggregationType.COUNT, "(*)", null);
aggregationSelectItem1.setIndex(1);
......@@ -86,12 +81,11 @@ public final class GroupByMemoryMergedResultTest {
AggregationProjection derivedAggregationSelectItem2 = new AggregationProjection(AggregationType.SUM, "(num)", "AVG_DERIVED_SUM_0");
aggregationSelectItem2.setIndex(5);
aggregationSelectItem2.getDerivedAggregationProjections().add(derivedAggregationSelectItem2);
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, false, Arrays.<Projection>asList(aggregationSelectItem1, aggregationSelectItem2));
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, false, Arrays.<Projection>asList(aggregationSelectItem1, aggregationSelectItem2), Collections.<String>emptyList());
selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.singletonList(createOrderByItem(new IndexOrderByItemSegment(0, 0, 3, OrderDirection.ASC, OrderDirection.ASC))), 0),
new OrderByContext(Collections.singletonList(createOrderByItem(new IndexOrderByItemSegment(0, 0, 3, OrderDirection.DESC, OrderDirection.ASC))), false),
projectionsContext, new PaginationContext(null, null, Collections.emptyList()));
routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
}
private ResultSet mockResultSet() throws SQLException {
......@@ -115,14 +109,14 @@ public final class GroupByMemoryMergedResultTest {
@Test
public void assertNextForResultSetsAllEmpty() throws SQLException {
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertFalse(actual.next());
}
@Test
public void assertNextForSomeResultSetsEmpty() throws SQLException {
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
when(resultSets.get(0).next()).thenReturn(true, false);
when(resultSets.get(0).getObject(1)).thenReturn(20);
when(resultSets.get(0).getObject(2)).thenReturn(0);
......
......@@ -20,7 +20,7 @@ package org.apache.shardingsphere.core.merge.dql.groupby;
import com.google.common.collect.Lists;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.dql.common.MemoryQueryResultRow;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.Projection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.ProjectionsContext;
......@@ -59,7 +59,7 @@ public final class GroupByRowComparatorTest {
new OrderByContext(Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.ASC, OrderDirection.ASC))), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
GroupByRowComparator groupByRowComparator = new GroupByRowComparator(selectSQLStatementContext, caseSensitives);
MemoryQueryResultRow o1 = new MemoryQueryResultRow(mockQueryResult("1", "2"));
MemoryQueryResultRow o2 = new MemoryQueryResultRow(mockQueryResult("3", "4"));
......@@ -75,7 +75,7 @@ public final class GroupByRowComparatorTest {
new OrderByContext(Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.DESC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.DESC, OrderDirection.ASC))), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
GroupByRowComparator groupByRowComparator = new GroupByRowComparator(selectSQLStatementContext, caseSensitives);
MemoryQueryResultRow o1 = new MemoryQueryResultRow(mockQueryResult("1", "2"));
MemoryQueryResultRow o2 = new MemoryQueryResultRow(mockQueryResult("3", "4"));
......@@ -91,7 +91,7 @@ public final class GroupByRowComparatorTest {
new OrderByContext(Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.DESC, OrderDirection.ASC))), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
GroupByRowComparator groupByRowComparator = new GroupByRowComparator(selectSQLStatementContext, caseSensitives);
MemoryQueryResultRow o1 = new MemoryQueryResultRow(mockQueryResult("1", "2"));
MemoryQueryResultRow o2 = new MemoryQueryResultRow(mockQueryResult("1", "2"));
......@@ -104,7 +104,7 @@ public final class GroupByRowComparatorTest {
new GroupByContext(Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.ASC, OrderDirection.ASC))), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
GroupByRowComparator groupByRowComparator = new GroupByRowComparator(selectSQLStatementContext, caseSensitives);
MemoryQueryResultRow o1 = new MemoryQueryResultRow(mockQueryResult("1", "2"));
MemoryQueryResultRow o2 = new MemoryQueryResultRow(mockQueryResult("3", "4"));
......@@ -117,7 +117,7 @@ public final class GroupByRowComparatorTest {
new GroupByContext(Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.DESC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.DESC, OrderDirection.ASC))), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
GroupByRowComparator groupByRowComparator = new GroupByRowComparator(selectSQLStatementContext, caseSensitives);
MemoryQueryResultRow o1 = new MemoryQueryResultRow(mockQueryResult("1", "2"));
MemoryQueryResultRow o2 = new MemoryQueryResultRow(mockQueryResult("3", "4"));
......@@ -130,7 +130,7 @@ public final class GroupByRowComparatorTest {
new GroupByContext(Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.DESC, OrderDirection.ASC))), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
GroupByRowComparator groupByRowComparator = new GroupByRowComparator(selectSQLStatementContext, caseSensitives);
MemoryQueryResultRow o1 = new MemoryQueryResultRow(mockQueryResult("1", "2"));
MemoryQueryResultRow o2 = new MemoryQueryResultRow(mockQueryResult("1", "2"));
......@@ -152,6 +152,6 @@ public final class GroupByRowComparatorTest {
for (Object each : values) {
when(result.getObject(++index)).thenReturn(each);
}
return new TestQueryResult(result);
return new ResultSetBasedQueryResultFixture(result);
}
}
......@@ -22,8 +22,9 @@ import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.MergedResult;
import org.apache.shardingsphere.core.merge.dql.DQLMergeEngine;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
......@@ -31,13 +32,7 @@ import org.apache.shardingsphere.sql.parser.relation.segment.select.pagination.P
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.Projection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.ProjectionsContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationProjection;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
import org.apache.shardingsphere.core.route.SQLRouteResult;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.IndexOrderByItemSegment;
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
import org.junit.Before;
......@@ -69,14 +64,14 @@ public final class GroupByStreamMergedResultTest {
private List<QueryResult> queryResults;
private SQLRouteResult routeResult;
private SelectSQLStatementContext selectSQLStatementContext;
@Before
public void setUp() throws SQLException {
resultSets = Lists.newArrayList(mockResultSet(), mockResultSet(), mockResultSet());
queryResults = new ArrayList<>(resultSets.size());
for (ResultSet each : resultSets) {
queryResults.add(new TestQueryResult(each));
queryResults.add(new ResultSetBasedQueryResultFixture(each));
}
AggregationProjection aggregationSelectItem1 = new AggregationProjection(AggregationType.COUNT, "(*)", null);
aggregationSelectItem1.setIndex(1);
......@@ -88,12 +83,11 @@ public final class GroupByStreamMergedResultTest {
AggregationProjection derivedAggregationSelectItem2 = new AggregationProjection(AggregationType.SUM, "(num)", "AVG_DERIVED_SUM_0");
aggregationSelectItem2.setIndex(6);
aggregationSelectItem2.getDerivedAggregationProjections().add(derivedAggregationSelectItem2);
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, false, Arrays.<Projection>asList(aggregationSelectItem1, aggregationSelectItem2));
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, false, Arrays.<Projection>asList(aggregationSelectItem1, aggregationSelectItem2), Collections.<String>emptyList());
selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.singletonList(new OrderByItem(new IndexOrderByItemSegment(0, 0, 3, OrderDirection.ASC, OrderDirection.ASC))), 0),
new OrderByContext(Collections.singletonList(new OrderByItem(new IndexOrderByItemSegment(0, 0, 3, OrderDirection.ASC, OrderDirection.ASC))), false),
projectionsContext, new PaginationContext(null, null, Collections.emptyList()));
routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
}
private ResultSet mockResultSet() throws SQLException {
......@@ -112,14 +106,14 @@ public final class GroupByStreamMergedResultTest {
@Test
public void assertNextForResultSetsAllEmpty() throws SQLException {
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertFalse(actual.next());
}
@Test
public void assertNextForSomeResultSetsEmpty() throws SQLException {
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
when(resultSets.get(0).next()).thenReturn(true, false);
when(resultSets.get(0).getObject(1)).thenReturn(20);
when(resultSets.get(0).getObject(2)).thenReturn(0);
......@@ -161,7 +155,7 @@ public final class GroupByStreamMergedResultTest {
@Test
public void assertNextForMix() throws SQLException {
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
when(resultSets.get(0).next()).thenReturn(true, false);
when(resultSets.get(0).getObject(1)).thenReturn(20);
when(resultSets.get(0).getObject(2)).thenReturn(0);
......
......@@ -17,7 +17,7 @@
package org.apache.shardingsphere.core.merge.dql.groupby;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.IndexOrderByItemSegment;
......@@ -50,7 +50,7 @@ public final class GroupByValueTest {
@Test
public void assertGetGroupByValues() throws SQLException {
List<?> actual = new GroupByValue(new TestQueryResult(resultSet),
List<?> actual = new GroupByValue(new ResultSetBasedQueryResultFixture(resultSet),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 3, OrderDirection.DESC, OrderDirection.ASC)))).getGroupValues();
......@@ -60,11 +60,11 @@ public final class GroupByValueTest {
@Test
public void assertGroupByValueEquals() throws SQLException {
GroupByValue groupByValue1 = new GroupByValue(new TestQueryResult(resultSet),
GroupByValue groupByValue1 = new GroupByValue(new ResultSetBasedQueryResultFixture(resultSet),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 3, OrderDirection.DESC, OrderDirection.ASC))));
GroupByValue groupByValue2 = new GroupByValue(new TestQueryResult(resultSet),
GroupByValue groupByValue2 = new GroupByValue(new ResultSetBasedQueryResultFixture(resultSet),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 3, OrderDirection.DESC, OrderDirection.ASC))));
......@@ -75,11 +75,11 @@ public final class GroupByValueTest {
@Test
public void assertGroupByValueNotEquals() throws SQLException {
GroupByValue groupByValue1 = new GroupByValue(new TestQueryResult(resultSet),
GroupByValue groupByValue1 = new GroupByValue(new ResultSetBasedQueryResultFixture(resultSet),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 3, OrderDirection.DESC, OrderDirection.ASC))));
GroupByValue groupByValue2 = new GroupByValue(new TestQueryResult(resultSet),
GroupByValue groupByValue2 = new GroupByValue(new ResultSetBasedQueryResultFixture(resultSet),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 3, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.DESC, OrderDirection.ASC))));
......
......@@ -27,18 +27,33 @@ public final class AggregationUnitFactoryTest {
@Test
public void assertCreateComparableAggregationUnit() {
assertThat(AggregationUnitFactory.create(AggregationType.MIN), instanceOf(ComparableAggregationUnit.class));
assertThat(AggregationUnitFactory.create(AggregationType.MAX), instanceOf(ComparableAggregationUnit.class));
assertThat(AggregationUnitFactory.create(AggregationType.MIN, false), instanceOf(ComparableAggregationUnit.class));
assertThat(AggregationUnitFactory.create(AggregationType.MAX, false), instanceOf(ComparableAggregationUnit.class));
}
@Test
public void assertCreateAccumulationAggregationUnit() {
assertThat(AggregationUnitFactory.create(AggregationType.SUM), instanceOf(AccumulationAggregationUnit.class));
assertThat(AggregationUnitFactory.create(AggregationType.COUNT), instanceOf(AccumulationAggregationUnit.class));
assertThat(AggregationUnitFactory.create(AggregationType.SUM, false), instanceOf(AccumulationAggregationUnit.class));
assertThat(AggregationUnitFactory.create(AggregationType.COUNT, false), instanceOf(AccumulationAggregationUnit.class));
}
@Test
public void assertCreateAverageAggregationUnit() {
assertThat(AggregationUnitFactory.create(AggregationType.AVG), instanceOf(AverageAggregationUnit.class));
assertThat(AggregationUnitFactory.create(AggregationType.AVG, false), instanceOf(AverageAggregationUnit.class));
}
@Test
public void assertCreateDistinctSumAggregationUnit() {
assertThat(AggregationUnitFactory.create(AggregationType.SUM, true), instanceOf(DistinctSumAggregationUnit.class));
}
@Test
public void assertCreateDistinctCountAggregationUnit() {
assertThat(AggregationUnitFactory.create(AggregationType.COUNT, true), instanceOf(DistinctCountAggregationUnit.class));
}
@Test
public void assertCreateDistinctAverageAggregationUnit() {
assertThat(AggregationUnitFactory.create(AggregationType.AVG, true), instanceOf(DistinctAverageAggregationUnit.class));
}
}
......@@ -22,8 +22,7 @@ import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.MergedResult;
import org.apache.shardingsphere.core.merge.dql.DQLMergeEngine;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
......@@ -31,9 +30,6 @@ import org.apache.shardingsphere.sql.parser.relation.segment.select.pagination.P
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.Projection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.ProjectionsContext;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
import org.apache.shardingsphere.core.route.SQLRouteResult;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
import org.junit.Before;
import org.junit.Test;
......@@ -55,23 +51,23 @@ public final class IteratorStreamMergedResultTest {
private List<QueryResult> queryResults;
private SQLRouteResult routeResult;
private SelectSQLStatementContext selectSQLStatementContext;
@Before
public void setUp() throws SQLException {
ResultSet resultSet = mock(ResultSet.class);
ResultSetMetaData resultSetMetaData = mock(ResultSetMetaData.class);
when(resultSet.getMetaData()).thenReturn(resultSetMetaData);
queryResults = Lists.<QueryResult>newArrayList(new TestQueryResult(resultSet), new TestQueryResult(mock(ResultSet.class)), new TestQueryResult(mock(ResultSet.class)));
routeResult = new SQLRouteResult(new SelectSQLStatementContext(new SelectStatement(),
queryResults = Lists.<QueryResult>newArrayList(
new ResultSetBasedQueryResultFixture(resultSet), new ResultSetBasedQueryResultFixture(mock(ResultSet.class)), new ResultSetBasedQueryResultFixture(mock(ResultSet.class)));
selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList())),
new ShardingConditions(Collections.<ShardingCondition>emptyList()));
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
}
@Test
public void assertNextForResultSetsAllEmpty() throws SQLException {
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertFalse(actual.next());
}
......@@ -81,7 +77,7 @@ public final class IteratorStreamMergedResultTest {
for (QueryResult each : queryResults) {
when(each.next()).thenReturn(true, false);
}
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertTrue(actual.next());
......@@ -92,7 +88,7 @@ public final class IteratorStreamMergedResultTest {
@Test
public void assertNextForFirstResultSetsNotEmptyOnly() throws SQLException {
when(queryResults.get(0).next()).thenReturn(true, false);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertFalse(actual.next());
......@@ -101,7 +97,7 @@ public final class IteratorStreamMergedResultTest {
@Test
public void assertNextForMiddleResultSetsNotEmpty() throws SQLException {
when(queryResults.get(1).next()).thenReturn(true, false);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertFalse(actual.next());
......@@ -110,7 +106,7 @@ public final class IteratorStreamMergedResultTest {
@Test
public void assertNextForLastResultSetsNotEmptyOnly() throws SQLException {
when(queryResults.get(2).next()).thenReturn(true, false);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertFalse(actual.next());
......@@ -118,13 +114,13 @@ public final class IteratorStreamMergedResultTest {
@Test
public void assertNextForMix() throws SQLException {
queryResults.add(new TestQueryResult(mock(ResultSet.class)));
queryResults.add(new TestQueryResult(mock(ResultSet.class)));
queryResults.add(new TestQueryResult(mock(ResultSet.class)));
queryResults.add(new ResultSetBasedQueryResultFixture(mock(ResultSet.class)));
queryResults.add(new ResultSetBasedQueryResultFixture(mock(ResultSet.class)));
queryResults.add(new ResultSetBasedQueryResultFixture(mock(ResultSet.class)));
when(queryResults.get(1).next()).thenReturn(true, false);
when(queryResults.get(3).next()).thenReturn(true, false);
when(queryResults.get(5).next()).thenReturn(true, false);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertTrue(actual.next());
......
......@@ -22,20 +22,15 @@ import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.MergedResult;
import org.apache.shardingsphere.core.merge.dql.DQLMergeEngine;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
import org.apache.shardingsphere.sql.parser.relation.segment.select.pagination.PaginationContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.Projection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.ProjectionsContext;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
import org.apache.shardingsphere.core.route.SQLRouteResult;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.IndexOrderByItemSegment;
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
import org.junit.Before;
......@@ -60,17 +55,16 @@ public final class OrderByStreamMergedResultTest {
private List<QueryResult> queryResults;
private SQLRouteResult routeResult;
private SelectSQLStatementContext selectSQLStatementContext;
@Before
public void setUp() throws SQLException {
queryResults = Lists.<QueryResult>newArrayList(
new TestQueryResult(mockResultSet()), new TestQueryResult(mockResultSet()), new TestQueryResult(mockResultSet()));
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new ResultSetBasedQueryResultFixture(mockResultSet()), new ResultSetBasedQueryResultFixture(mockResultSet()), new ResultSetBasedQueryResultFixture(mockResultSet()));
selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0),
new OrderByContext(Collections.singletonList(new OrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC))), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
}
private ResultSet mockResultSet() throws SQLException {
......@@ -82,14 +76,14 @@ public final class OrderByStreamMergedResultTest {
@Test
public void assertNextForResultSetsAllEmpty() throws SQLException {
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertFalse(actual.next());
}
@Test
public void assertNextForSomeResultSetsEmpty() throws SQLException {
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
when(queryResults.get(0).next()).thenReturn(true, false);
when(queryResults.get(0).getValue(1, Object.class)).thenReturn("2");
when(queryResults.get(2).next()).thenReturn(true, true, false);
......@@ -106,7 +100,7 @@ public final class OrderByStreamMergedResultTest {
@Test
public void assertNextForMix() throws SQLException {
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
when(queryResults.get(0).next()).thenReturn(true, false);
when(queryResults.get(0).getValue(1, Object.class)).thenReturn("2");
when(queryResults.get(1).next()).thenReturn(true, true, true, false);
......@@ -140,7 +134,7 @@ public final class OrderByStreamMergedResultTest {
when(queryResults.get(2).next()).thenReturn(true, false);
when(queryResults.get(2).isCaseSensitive(1)).thenReturn(true);
when(queryResults.get(2).getValue(1, Object.class)).thenReturn("A");
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertThat(actual.getValue(1, Object.class).toString(), is("A"));
......@@ -164,7 +158,7 @@ public final class OrderByStreamMergedResultTest {
when(queryResults.get(2).next()).thenReturn(true, false);
when(queryResults.get(2).isCaseSensitive(1)).thenReturn(false);
when(queryResults.get(2).getValue(1, Object.class)).thenReturn("A");
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertThat(actual.getValue(1, Object.class).toString(), is("a"));
......
......@@ -17,7 +17,7 @@
package org.apache.shardingsphere.core.merge.dql.orderby;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.IndexOrderByItemSegment;
......@@ -64,14 +64,14 @@ public final class OrderByValueTest {
@Test
public void assertCompareToForAsc() throws SQLException {
OrderByValue orderByValue1 = new OrderByValue(new TestQueryResult(resultSet1),
OrderByValue orderByValue1 = new OrderByValue(new ResultSetBasedQueryResultFixture(resultSet1),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.ASC, OrderDirection.ASC))));
assertTrue(orderByValue1.next());
when(resultSet2.getObject(1)).thenReturn("3");
when(resultSet2.getObject(2)).thenReturn("4");
OrderByValue orderByValue2 = new OrderByValue(new TestQueryResult(resultSet2),
OrderByValue orderByValue2 = new OrderByValue(new ResultSetBasedQueryResultFixture(resultSet2),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.ASC, OrderDirection.ASC))));
......@@ -83,14 +83,14 @@ public final class OrderByValueTest {
@Test
public void assertCompareToForDesc() throws SQLException {
OrderByValue orderByValue1 = new OrderByValue(new TestQueryResult(resultSet1),
OrderByValue orderByValue1 = new OrderByValue(new ResultSetBasedQueryResultFixture(resultSet1),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.DESC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.DESC, OrderDirection.ASC))));
assertTrue(orderByValue1.next());
when(resultSet2.getObject(1)).thenReturn("3");
when(resultSet2.getObject(2)).thenReturn("4");
OrderByValue orderByValue2 = new OrderByValue(new TestQueryResult(resultSet2),
OrderByValue orderByValue2 = new OrderByValue(new ResultSetBasedQueryResultFixture(resultSet2),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.DESC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.DESC, OrderDirection.ASC))));
......@@ -102,14 +102,15 @@ public final class OrderByValueTest {
@Test
public void assertCompareToWhenEqual() throws SQLException {
OrderByValue orderByValue1 = new OrderByValue(new TestQueryResult(resultSet1),
OrderByValue orderByValue1 = new OrderByValue(new ResultSetBasedQueryResultFixture(resultSet1),
Arrays.asList(
createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.DESC, OrderDirection.ASC))));
assertTrue(orderByValue1.next());
when(resultSet2.getObject(1)).thenReturn("1");
when(resultSet2.getObject(2)).thenReturn("2");
OrderByValue orderByValue2 = new OrderByValue(new TestQueryResult(resultSet2), Arrays.asList(createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
OrderByValue orderByValue2 = new OrderByValue(new ResultSetBasedQueryResultFixture(resultSet2),
Arrays.asList(createOrderByItem(new IndexOrderByItemSegment(0, 0, 1, OrderDirection.ASC, OrderDirection.ASC)),
createOrderByItem(new IndexOrderByItemSegment(0, 0, 2, OrderDirection.DESC, OrderDirection.ASC))));
assertTrue(orderByValue2.next());
assertThat(orderByValue1.compareTo(orderByValue2), is(0));
......
......@@ -22,19 +22,14 @@ import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.MergedResult;
import org.apache.shardingsphere.core.merge.dql.DQLMergeEngine;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
import org.apache.shardingsphere.sql.parser.relation.segment.select.pagination.PaginationContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.Projection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.ProjectionsContext;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
import org.apache.shardingsphere.core.route.SQLRouteResult;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
import org.junit.Before;
......@@ -69,30 +64,28 @@ public final class LimitDecoratorMergedResultTest {
}
queryResults = new ArrayList<>(resultSets.size());
for (ResultSet each : resultSets) {
queryResults.add(new TestQueryResult(each));
queryResults.add(new ResultSetBasedQueryResultFixture(each));
}
}
@Test
public void assertNextForSkipAll() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralLimitValueSegment(0, 0, Integer.MAX_VALUE), null, Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertFalse(actual.next());
}
@Test
public void assertNextWithoutRowCount() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralLimitValueSegment(0, 0, 2), null, Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
for (int i = 0; i < 6; i++) {
assertTrue(actual.next());
......@@ -102,12 +95,11 @@ public final class LimitDecoratorMergedResultTest {
@Test
public void assertNextWithRowCount() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralLimitValueSegment(0, 0, 2), new NumberLiteralLimitValueSegment(0, 0, 2), Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("MySQL"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertTrue(actual.next());
......
......@@ -22,19 +22,14 @@ import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.MergedResult;
import org.apache.shardingsphere.core.merge.dql.DQLMergeEngine;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
import org.apache.shardingsphere.sql.parser.relation.segment.select.pagination.PaginationContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.Projection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.ProjectionsContext;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
import org.apache.shardingsphere.core.route.SQLRouteResult;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.pagination.rownum.NumberLiteralRowNumberValueSegment;
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
import org.junit.Before;
......@@ -69,29 +64,27 @@ public final class RowNumberDecoratorMergedResultTest {
}
queryResults = new ArrayList<>(resultSets.size());
for (ResultSet each : resultSets) {
queryResults.add(new TestQueryResult(each));
queryResults.add(new ResultSetBasedQueryResultFixture(each));
}
}
@Test
public void assertNextForSkipAll() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralRowNumberValueSegment(0, 0, Integer.MAX_VALUE, true), null, Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("Oracle"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("Oracle"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertFalse(actual.next());
}
@Test
public void assertNextWithoutOffsetWithoutRowCount() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("Oracle"), mock(TableMetas.class), routeResult, queryResults);
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()), new PaginationContext(null, null, Collections.emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("Oracle"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
for (int i = 0; i < 8; i++) {
assertTrue(actual.next());
......@@ -101,12 +94,11 @@ public final class RowNumberDecoratorMergedResultTest {
@Test
public void assertNextForRowCountBoundOpenedFalse() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralRowNumberValueSegment(0, 0, 2, true), new NumberLiteralRowNumberValueSegment(0, 0, 4, false), Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("Oracle"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("Oracle"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertTrue(actual.next());
......@@ -115,12 +107,11 @@ public final class RowNumberDecoratorMergedResultTest {
@Test
public void assertNextForRowCountBoundOpenedTrue() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralRowNumberValueSegment(0, 0, 2, true), new NumberLiteralRowNumberValueSegment(0, 0, 4, true), Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("Oracle"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("Oracle"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertTrue(actual.next());
......
......@@ -22,19 +22,14 @@ import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.merge.MergedResult;
import org.apache.shardingsphere.core.merge.dql.DQLMergeEngine;
import org.apache.shardingsphere.core.merge.fixture.TestQueryResult;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.core.merge.fixture.ResultSetBasedQueryResultFixture;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
import org.apache.shardingsphere.sql.parser.relation.segment.select.pagination.PaginationContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.Projection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.ProjectionsContext;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
import org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext;
import org.apache.shardingsphere.core.route.SQLRouteResult;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.pagination.rownum.NumberLiteralRowNumberValueSegment;
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
......@@ -70,30 +65,28 @@ public final class TopAndRowNumberDecoratorMergedResultTest {
}
queryResults = new ArrayList<>(resultSets.size());
for (ResultSet each : resultSets) {
queryResults.add(new TestQueryResult(each));
queryResults.add(new ResultSetBasedQueryResultFixture(each));
}
}
@Test
public void assertNextForSkipAll() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralRowNumberValueSegment(0, 0, Integer.MAX_VALUE, true), null, Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertFalse(actual.next());
}
@Test
public void assertNextWithoutOffsetWithRowCount() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(null, new NumberLiteralLimitValueSegment(0, 0, 5), Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
for (int i = 0; i < 5; i++) {
assertTrue(actual.next());
......@@ -103,12 +96,11 @@ public final class TopAndRowNumberDecoratorMergedResultTest {
@Test
public void assertNextWithOffsetWithoutRowCount() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralRowNumberValueSegment(0, 0, 2, true), null, Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
for (int i = 0; i < 7; i++) {
assertTrue(actual.next());
......@@ -118,12 +110,11 @@ public final class TopAndRowNumberDecoratorMergedResultTest {
@Test
public void assertNextWithOffsetBoundOpenedFalse() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralRowNumberValueSegment(0, 0, 2, false), new NumberLiteralLimitValueSegment(0, 0, 4), Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertTrue(actual.next());
......@@ -132,12 +123,11 @@ public final class TopAndRowNumberDecoratorMergedResultTest {
@Test
public void assertNextWithOffsetBoundOpenedTrue() throws SQLException {
SQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(new NumberLiteralRowNumberValueSegment(0, 0, 2, true), new NumberLiteralLimitValueSegment(0, 0, 4), Collections.emptyList()));
SQLRouteResult routeResult = new SQLRouteResult(selectSQLStatementContext, new ShardingConditions(Collections.<ShardingCondition>emptyList()));
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), mock(TableMetas.class), routeResult, queryResults);
mergeEngine = new DQLMergeEngine(DatabaseTypes.getActualDatabaseType("SQLServer"), selectSQLStatementContext, queryResults);
MergedResult actual = mergeEngine.merge();
assertTrue(actual.next());
assertTrue(actual.next());
......
......@@ -19,87 +19,75 @@ package org.apache.shardingsphere.core.merge.fixture;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResultMetaData;
import java.io.InputStream;
import java.sql.SQLException;
import java.util.Calendar;
import java.util.Iterator;
/**
* fixture for test DescribeTableMergedResult.
*
* @author liya
*/
@RequiredArgsConstructor
public final class DescribeQueryResultFixture implements QueryResult {
public final class QueryResultBasedQueryResultFixture implements QueryResult {
private final Iterator<QueryResult> queryResults;
private QueryResult currQueryResult;
@Override
public boolean next() throws SQLException {
public boolean next() {
boolean hasNext = queryResults.hasNext();
if (hasNext) {
currQueryResult = queryResults.next();
}
return hasNext;
}
@Override
public Object getValue(final int columnIndex, final Class<?> type) throws SQLException {
return currQueryResult.getValue(columnIndex, type);
}
@Override
public Object getValue(final String columnLabel, final Class<?> type) throws SQLException {
public Object getValue(final String columnLabel, final Class<?> type) {
return null;
}
@Override
public Object getCalendarValue(final int columnIndex, final Class<?> type, final Calendar calendar) throws SQLException {
public Object getCalendarValue(final int columnIndex, final Class<?> type, final Calendar calendar) {
return null;
}
@Override
public Object getCalendarValue(final String columnLabel, final Class<?> type, final Calendar calendar) throws SQLException {
public Object getCalendarValue(final String columnLabel, final Class<?> type, final Calendar calendar) {
return null;
}
@Override
public InputStream getInputStream(final int columnIndex, final String type) throws SQLException {
public InputStream getInputStream(final int columnIndex, final String type) {
return null;
}
@Override
public InputStream getInputStream(final String columnLabel, final String type) throws SQLException {
public InputStream getInputStream(final String columnLabel, final String type) {
return null;
}
@Override
public boolean wasNull() throws SQLException {
public boolean wasNull() {
return false;
}
@Override
public boolean isCaseSensitive(final int columnIndex) throws SQLException {
public boolean isCaseSensitive(final int columnIndex) {
return false;
}
@Override
public QueryResultMetaData getQueryResultMetaData() {
return null;
}
@Override
public int getColumnCount() throws SQLException {
return currQueryResult.getColumnCount();
}
@Override
public String getColumnLabel(final int columnIndex) throws SQLException {
public String getColumnLabel(final int columnIndex) {
return null;
}
}
......@@ -19,7 +19,6 @@ package org.apache.shardingsphere.core.merge.fixture;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResultMetaData;
import java.io.InputStream;
import java.io.Reader;
......@@ -36,7 +35,7 @@ import java.sql.Timestamp;
import java.util.Calendar;
@RequiredArgsConstructor
public final class TestQueryResult implements QueryResult {
public final class ResultSetBasedQueryResultFixture implements QueryResult {
private final ResultSet resultSet;
......@@ -206,17 +205,7 @@ public final class TestQueryResult implements QueryResult {
public boolean isCaseSensitive(final int columnIndex) throws SQLException {
return resultSet.getMetaData().isCaseSensitive(columnIndex);
}
@Override
public QueryResultMetaData getQueryResultMetaData() {
try {
return new QueryResultMetaData(resultSet.getMetaData());
} catch (SQLException e) {
e.printStackTrace();
}
return null;
}
@Override
public int getColumnCount() throws SQLException {
return resultSet.getMetaData().getColumnCount();
......
......@@ -48,7 +48,7 @@ public final class ComplexRoutingEngineTest extends AbstractRoutingEngineTest {
ShardingRule shardingRule = createBindingShardingRule();
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(null, null, Collections.emptyList()));
ComplexRoutingEngine complexRoutingEngine = new ComplexRoutingEngine(shardingRule, Arrays.asList("t_order", "t_order_item"), selectSQLStatementContext, createShardingConditions("t_order"));
RoutingResult routingResult = complexRoutingEngine.route();
......@@ -66,7 +66,7 @@ public final class ComplexRoutingEngineTest extends AbstractRoutingEngineTest {
ShardingRule shardingRule = createBroadcastShardingRule();
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(null, null, Collections.emptyList()));
ComplexRoutingEngine complexRoutingEngine = new ComplexRoutingEngine(shardingRule, Arrays.asList("t_order", "t_config"), selectSQLStatementContext, createShardingConditions("t_order"));
RoutingResult routingResult = complexRoutingEngine.route();
......
......@@ -192,7 +192,7 @@ public final class StandardRoutingEngineTest extends AbstractRoutingEngineTest {
private StandardRoutingEngine createStandardRoutingEngine(final ShardingRule shardingRule, final String logicTableName, final ShardingConditions shardingConditions) {
return new StandardRoutingEngine(shardingRule, logicTableName, new SelectSQLStatementContext(new SelectStatement(),
new GroupByContext(Collections.<OrderByItem>emptyList(), 0), new OrderByContext(Collections.<OrderByItem>emptyList(), false),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList()),
new ProjectionsContext(0, 0, false, Collections.<Projection>emptyList(), Collections.<String>emptyList()),
new PaginationContext(null, null, Collections.emptyList())), shardingConditions);
}
}
......@@ -25,9 +25,9 @@ import lombok.Getter;
import lombok.Setter;
import org.apache.shardingsphere.core.constant.properties.ShardingProperties;
import org.apache.shardingsphere.core.constant.properties.ShardingPropertiesConstant;
import org.apache.shardingsphere.core.execute.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.metadata.TableMetaDataInitializer;
import org.apache.shardingsphere.core.execute.sql.execute.SQLExecuteCallback;
import org.apache.shardingsphere.core.execute.sql.execute.SQLExecuteTemplate;
......
......@@ -25,8 +25,8 @@ import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import lombok.Getter;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.execute.SQLExecuteCallback;
import org.apache.shardingsphere.core.execute.sql.execute.threadlocal.ExecutorExceptionHandler;
import org.apache.shardingsphere.core.execute.sql.prepare.SQLExecutePrepareCallback;
......
......@@ -20,8 +20,8 @@ package org.apache.shardingsphere.shardingjdbc.executor;
import lombok.Getter;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.constant.properties.ShardingProperties;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.execute.SQLExecuteCallback;
import org.apache.shardingsphere.core.execute.sql.execute.result.MemoryQueryResult;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
......
......@@ -19,8 +19,8 @@ package org.apache.shardingsphere.shardingjdbc.executor;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.constant.properties.ShardingProperties;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.execute.SQLExecuteCallback;
import org.apache.shardingsphere.core.execute.sql.execute.result.MemoryQueryResult;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
......
......@@ -21,7 +21,7 @@ import lombok.Getter;
import org.apache.shardingsphere.core.constant.properties.ShardingProperties;
import org.apache.shardingsphere.core.constant.properties.ShardingPropertiesConstant;
import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteEngine;
import org.apache.shardingsphere.core.rule.BaseRule;
import org.apache.shardingsphere.core.util.ConfigurationLogger;
import org.apache.shardingsphere.spi.database.DatabaseType;
......
......@@ -18,7 +18,7 @@
package org.apache.shardingsphere.shardingjdbc.jdbc.core.context;
import org.apache.shardingsphere.core.constant.properties.ShardingProperties;
import org.apache.shardingsphere.core.execute.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteEngine;
import org.apache.shardingsphere.sql.parser.SQLParseEngine;
import org.apache.shardingsphere.core.rule.BaseRule;
import org.apache.shardingsphere.spi.database.DatabaseType;
......
......@@ -23,7 +23,7 @@ import lombok.Getter;
import org.apache.shardingsphere.core.constant.properties.ShardingProperties;
import org.apache.shardingsphere.core.constant.properties.ShardingPropertiesConstant;
import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.sql.execute.threadlocal.ExecutorExceptionHandler;
import org.apache.shardingsphere.sql.parser.relation.segment.table.TablesContext;
import org.apache.shardingsphere.sql.parser.relation.statement.SQLStatementContext;
......
......@@ -19,8 +19,8 @@ package org.apache.shardingsphere.shardingjdbc.executor;
import lombok.SneakyThrows;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.route.BatchRouteUnit;
import org.apache.shardingsphere.core.route.RouteUnit;
import org.apache.shardingsphere.core.route.SQLUnit;
......@@ -121,7 +121,7 @@ public final class BatchPreparedStatementExecutorTest extends AbstractBaseExecut
verify(preparedStatement2).executeBatch();
}
private void setExecuteGroups(final List<PreparedStatement> preparedStatements) throws SQLException {
private void setExecuteGroups(final List<PreparedStatement> preparedStatements) {
Collection<ShardingExecuteGroup<StatementExecuteUnit>> executeGroups = new LinkedList<>();
List<StatementExecuteUnit> preparedStatementExecuteUnits = new LinkedList<>();
executeGroups.add(new ShardingExecuteGroup<>(preparedStatementExecuteUnits));
......
......@@ -19,8 +19,8 @@ package org.apache.shardingsphere.shardingjdbc.executor;
import lombok.SneakyThrows;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.route.RouteUnit;
import org.apache.shardingsphere.core.route.SQLUnit;
......
......@@ -20,7 +20,7 @@ package org.apache.shardingsphere.shardingjdbc.executor;
import com.google.common.collect.Lists;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.execute.SQLExecuteCallback;
import org.apache.shardingsphere.core.route.RouteUnit;
import org.apache.shardingsphere.core.route.SQLUnit;
......
......@@ -19,8 +19,8 @@ package org.apache.shardingsphere.shardingjdbc.executor;
import lombok.SneakyThrows;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.execute.result.QueryResult;
import org.apache.shardingsphere.core.execute.sql.execute.threadlocal.ExecutorExceptionHandler;
import org.apache.shardingsphere.core.route.RouteUnit;
......
......@@ -15,4 +15,4 @@
# limitations under the License.
#
mock-maker-inline
\ No newline at end of file
mock-maker-inline
......@@ -19,7 +19,7 @@ package org.apache.shardingsphere.opentracing.hook;
import io.opentracing.ActiveSpan;
import io.opentracing.tag.Tags;
import org.apache.shardingsphere.core.execute.ShardingExecuteDataMap;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteDataMap;
import org.apache.shardingsphere.core.execute.hook.RootInvokeHook;
import org.apache.shardingsphere.opentracing.ShardingTracer;
import org.apache.shardingsphere.opentracing.constant.ShardingTags;
......
......@@ -17,7 +17,7 @@
package org.apache.shardingsphere.opentracing.hook;
import org.apache.shardingsphere.core.execute.ShardingExecuteDataMap;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteDataMap;
import org.apache.shardingsphere.core.execute.hook.RootInvokeHook;
import org.apache.shardingsphere.core.execute.hook.SPIRootInvokeHook;
import org.apache.shardingsphere.spi.NewInstanceServiceLoader;
......
......@@ -21,7 +21,7 @@ import io.opentracing.ActiveSpan;
import io.opentracing.ActiveSpan.Continuation;
import io.opentracing.mock.MockSpan;
import io.opentracing.tag.Tags;
import org.apache.shardingsphere.core.execute.ShardingExecuteDataMap;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteDataMap;
import org.apache.shardingsphere.core.execute.hook.SPISQLExecutionHook;
import org.apache.shardingsphere.core.execute.hook.SQLExecutionHook;
import org.apache.shardingsphere.core.route.RouteUnit;
......
......@@ -19,8 +19,8 @@ package org.apache.shardingsphere.shardingproxy.backend.communication.jdbc.execu
import lombok.Getter;
import org.apache.shardingsphere.core.constant.properties.ShardingPropertiesConstant;
import org.apache.shardingsphere.core.execute.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteGroup;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.execute.SQLExecuteTemplate;
import org.apache.shardingsphere.core.execute.sql.execute.threadlocal.ExecutorExceptionHandler;
import org.apache.shardingsphere.core.execute.sql.prepare.SQLExecutePrepareTemplate;
......
......@@ -19,7 +19,7 @@ package org.apache.shardingsphere.shardingproxy.backend.communication.jdbc.execu
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.prepare.SQLExecutePrepareCallback;
import org.apache.shardingsphere.core.route.RouteUnit;
import org.apache.shardingsphere.spi.database.MySQLDatabaseType;
......
......@@ -21,7 +21,7 @@ import lombok.AccessLevel;
import lombok.Getter;
import lombok.NoArgsConstructor;
import org.apache.shardingsphere.core.constant.properties.ShardingPropertiesConstant;
import org.apache.shardingsphere.core.execute.ShardingExecuteEngine;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteEngine;
import org.apache.shardingsphere.shardingproxy.context.ShardingProxyContext;
/**
......
......@@ -18,7 +18,7 @@
package org.apache.shardingsphere.shardingproxy.backend.communication.jdbc.execute.callback;
import org.apache.shardingsphere.core.constant.ConnectionMode;
import org.apache.shardingsphere.core.execute.StatementExecuteUnit;
import org.apache.shardingsphere.core.execute.sql.StatementExecuteUnit;
import org.apache.shardingsphere.core.route.RouteUnit;
import org.apache.shardingsphere.core.route.SQLUnit;
import org.apache.shardingsphere.shardingproxy.backend.communication.jdbc.connection.BackendConnection;
......
......@@ -20,7 +20,7 @@ package org.apache.shardingsphere.shardingproxy.frontend.executor;
import com.google.common.util.concurrent.ListeningExecutorService;
import lombok.Getter;
import org.apache.shardingsphere.core.constant.properties.ShardingPropertiesConstant;
import org.apache.shardingsphere.core.execute.ShardingExecutorService;
import org.apache.shardingsphere.core.execute.engine.ShardingExecutorService;
import org.apache.shardingsphere.shardingproxy.context.ShardingProxyContext;
/**
......
......@@ -18,7 +18,7 @@
package org.apache.shardingsphere.transaction.base.seata.at;
import io.seata.core.context.RootContext;
import org.apache.shardingsphere.core.execute.ShardingExecuteDataMap;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteDataMap;
import java.util.Map;
......
......@@ -30,7 +30,7 @@ import io.seata.tm.api.GlobalTransactionContext;
import lombok.SneakyThrows;
import org.apache.commons.dbcp2.BasicDataSource;
import org.apache.shardingsphere.core.database.DatabaseTypes;
import org.apache.shardingsphere.core.execute.ShardingExecuteDataMap;
import org.apache.shardingsphere.core.execute.engine.ShardingExecuteDataMap;
import org.apache.shardingsphere.transaction.core.ResourceDataSource;
import org.apache.shardingsphere.transaction.core.TransactionType;
import org.junit.After;
......
......@@ -41,22 +41,4 @@ public final class SelectItemsSegment implements SQLSegment {
private final boolean distinctRow;
private final Collection<SelectItemSegment> selectItems = new LinkedList<>();
/**
* Find select item segments.
*
* @param selectItemSegmentType select item segment type
* @param <T> select item segment
* @return select item segments
*/
@SuppressWarnings("unchecked")
public <T extends SelectItemSegment> Collection<T> findSelectItemSegments(final Class<T> selectItemSegmentType) {
Collection<T> result = new LinkedList<>();
for (SelectItemSegment each : selectItems) {
if (each.getClass().equals(selectItemSegmentType)) {
result.add((T) each);
}
}
return result;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.sql.parser.sql.segment.dml.item;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import org.junit.Test;
import java.util.Collection;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertThat;
public final class SelectItemsSegmentTest {
@Test
public void assertFindSelectItemSegments() {
SelectItemsSegment selectItemsSegment = new SelectItemsSegment(0, 0, false);
SelectItemSegment selectItemSegment1 = new ExpressionSelectItemSegment(0, 0, "id");
selectItemsSegment.getSelectItems().add(selectItemSegment1);
SelectItemSegment selectItemSegment2 = new AggregationSelectItemSegment(0, 0, "count(*)", AggregationType.COUNT, 0);
selectItemsSegment.getSelectItems().add(selectItemSegment2);
Collection<AggregationSelectItemSegment> actual = selectItemsSegment.findSelectItemSegments(AggregationSelectItemSegment.class);
assertThat(actual.size(), is(1));
assertThat(actual.iterator().next(), is(selectItemSegment2));
}
}
......@@ -24,7 +24,6 @@ import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.OrderByIt
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedList;
/**
......@@ -42,7 +41,7 @@ public final class GroupByContextEngine {
*/
public GroupByContext createGroupByContext(final SelectStatement selectStatement) {
if (!selectStatement.getGroupBy().isPresent()) {
return new GroupByContext(Collections.<OrderByItem>emptyList(), 0);
return new GroupByContext(new LinkedList<OrderByItem>(), 0);
}
Collection<OrderByItem> groupByItems = new LinkedList<>();
for (OrderByItemSegment each : selectStatement.getGroupBy().get().getGroupByItems()) {
......
......@@ -21,15 +21,11 @@ import com.google.common.base.Optional;
import lombok.Getter;
import lombok.RequiredArgsConstructor;
import lombok.ToString;
import org.apache.shardingsphere.sql.parser.relation.metadata.RelationMetas;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationDistinctProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.ShorthandProjection;
import org.apache.shardingsphere.sql.parser.sql.segment.generic.TableSegment;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
......@@ -52,6 +48,8 @@ public final class ProjectionsContext {
private final Collection<Projection> projections;
private final List<String> columnLabels;
/**
* Judge is unqualified shorthand projection or not.
*
......@@ -128,45 +126,4 @@ public final class ProjectionsContext {
}
return result;
}
/**
* Get column labels.
*
* @param relationMetas relation metas
* @param tables tables
* @return column labels
*/
public List<String> getColumnLabels(final RelationMetas relationMetas, final Collection<TableSegment> tables) {
List<String> result = new ArrayList<>(projections.size());
for (Projection each : projections) {
if (each instanceof ShorthandProjection) {
result.addAll(getShorthandColumnLabels(relationMetas, tables, (ShorthandProjection) each));
} else {
result.add(each.getColumnLabel());
}
}
return result;
}
private Collection<String> getShorthandColumnLabels(final RelationMetas relationMetas, final Collection<TableSegment> tables, final ShorthandProjection shorthandProjection) {
return shorthandProjection.getOwner().isPresent()
? getQualifiedShorthandColumnLabels(relationMetas, tables, shorthandProjection.getOwner().get()) : getUnqualifiedShorthandColumnLabels(relationMetas, tables);
}
private Collection<String> getQualifiedShorthandColumnLabels(final RelationMetas relationMetas, final Collection<TableSegment> tables, final String owner) {
for (TableSegment each : tables) {
if (owner.equalsIgnoreCase(each.getAlias().or(each.getTableName()))) {
return relationMetas.getAllColumnNames(each.getTableName());
}
}
return Collections.emptyList();
}
private Collection<String> getUnqualifiedShorthandColumnLabels(final RelationMetas relationMetas, final Collection<TableSegment> tables) {
Collection<String> result = new LinkedList<>();
for (TableSegment each : tables) {
result.addAll(relationMetas.getAllColumnNames(each.getTableName()));
}
return result;
}
}
......@@ -37,10 +37,14 @@ import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.ColumnOrd
import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.IndexOrderByItemSegment;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.OrderByItemSegment;
import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.TextOrderByItemSegment;
import org.apache.shardingsphere.sql.parser.sql.segment.generic.TableSegment;
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
/**
* Projections context engine.
......@@ -66,11 +70,12 @@ public final class ProjectionsContextEngine {
*/
public ProjectionsContext createProjectionsContext(final String sql, final SelectStatement selectStatement, final GroupByContext groupByContext, final OrderByContext orderByContext) {
SelectItemsSegment selectItemsSegment = selectStatement.getSelectItems();
Collection<Projection> items = getProjections(sql, selectItemsSegment);
ProjectionsContext result = new ProjectionsContext(selectItemsSegment.getStartIndex(), selectItemsSegment.getStopIndex(), selectItemsSegment.isDistinctRow(), items);
Collection<Projection> projections = getProjections(sql, selectItemsSegment);
ProjectionsContext result = new ProjectionsContext(
selectItemsSegment.getStartIndex(), selectItemsSegment.getStopIndex(), selectItemsSegment.isDistinctRow(), projections, getColumnLabels(selectStatement.getTables(), projections));
TablesContext tablesContext = new TablesContext(selectStatement);
result.getProjections().addAll(getDerivedGroupByColumns(tablesContext, items, groupByContext));
result.getProjections().addAll(getDerivedOrderByColumns(tablesContext, items, orderByContext));
result.getProjections().addAll(getDerivedGroupByColumns(tablesContext, projections, groupByContext));
result.getProjections().addAll(getDerivedOrderByColumns(tablesContext, projections, orderByContext));
return result;
}
......@@ -85,6 +90,40 @@ public final class ProjectionsContextEngine {
return result;
}
private List<String> getColumnLabels(final Collection<TableSegment> tables, final Collection<Projection> projections) {
List<String> result = new ArrayList<>(projections.size());
for (Projection each : projections) {
if (each instanceof ShorthandProjection) {
result.addAll(getShorthandColumnLabels(tables, (ShorthandProjection) each));
} else {
result.add(each.getColumnLabel());
}
}
return result;
}
private Collection<String> getShorthandColumnLabels(final Collection<TableSegment> tables, final ShorthandProjection shorthandProjection) {
return shorthandProjection.getOwner().isPresent()
? getQualifiedShorthandColumnLabels(tables, shorthandProjection.getOwner().get()) : getUnqualifiedShorthandColumnLabels(tables);
}
private Collection<String> getQualifiedShorthandColumnLabels(final Collection<TableSegment> tables, final String owner) {
for (TableSegment each : tables) {
if (owner.equalsIgnoreCase(each.getAlias().or(each.getTableName()))) {
return relationMetas.getAllColumnNames(each.getTableName());
}
}
return Collections.emptyList();
}
private Collection<String> getUnqualifiedShorthandColumnLabels(final Collection<TableSegment> tables) {
Collection<String> result = new LinkedList<>();
for (TableSegment each : tables) {
result.addAll(relationMetas.getAllColumnNames(each.getTableName()));
}
return result;
}
private Collection<Projection> getDerivedGroupByColumns(final TablesContext tablesContext, final Collection<Projection> selectItems, final GroupByContext groupByContext) {
return getDerivedOrderColumns(tablesContext, selectItems, groupByContext.getItems(), DerivedColumn.GROUP_BY_ALIAS);
}
......
......@@ -159,16 +159,6 @@ public final class SelectSQLStatementContext extends CommonSQLStatementContext {
? ((ColumnOrderByItemSegment) orderByItemSegment).getColumn().getName() : ((ExpressionOrderByItemSegment) orderByItemSegment).getExpression();
}
/**
* Get column labels.
*
* @param relationMetas relation metas
* @return column labels
*/
public List<String> getColumnLabels(final RelationMetas relationMetas) {
return projectionsContext.getColumnLabels(relationMetas, ((SelectStatement) getSqlStatement()).getTables());
}
/**
* Judge group by and order by sequence is same or not.
*
......
......@@ -19,22 +19,15 @@ package org.apache.shardingsphere.sql.parser.relation.segment.select.projection;
import com.google.common.base.Optional;
import org.apache.shardingsphere.sql.parser.core.constant.AggregationType;
import org.apache.shardingsphere.sql.parser.relation.metadata.RelationMetaData;
import org.apache.shardingsphere.sql.parser.relation.metadata.RelationMetas;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationDistinctProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.AggregationProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.ColumnProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.DerivedProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.ExpressionProjection;
import org.apache.shardingsphere.sql.parser.relation.segment.select.projection.impl.ShorthandProjection;
import org.apache.shardingsphere.sql.parser.sql.segment.generic.TableSegment;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertFalse;
......@@ -45,46 +38,46 @@ public final class ProjectionsContextTest {
@Test
public void assertUnqualifiedShorthandProjectionWithEmptyItems() {
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.<Projection>emptySet());
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.<Projection>emptySet(), Collections.<String>emptyList());
assertFalse(projectionsContext.isUnqualifiedShorthandProjection());
}
@Test
public void assertUnqualifiedShorthandProjectionWithWrongSelectItem() {
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton((Projection) getColumnProjection()));
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton((Projection) getColumnProjection()), Collections.<String>emptyList());
assertFalse(projectionsContext.isUnqualifiedShorthandProjection());
}
@Test
public void assertUnqualifiedShorthandProjectionWithWrongShortSelectItem() {
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton((Projection) getShorthandProjection()));
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton((Projection) getShorthandProjection()), Collections.<String>emptyList());
assertFalse(projectionsContext.isUnqualifiedShorthandProjection());
}
@Test
public void assertUnqualifiedShorthandProjection() {
Projection projection = new ShorthandProjection(null);
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton(projection));
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton(projection), Collections.<String>emptyList());
assertTrue(projectionsContext.isUnqualifiedShorthandProjection());
}
@Test
public void assertFindAliasWithOutAlias() {
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.<Projection>emptyList());
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.<Projection>emptyList(), Collections.<String>emptyList());
assertFalse(projectionsContext.findAlias("").isPresent());
}
@Test
public void assertFindAlias() {
Projection projection = getColumnProjectionWithAlias();
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton(projection));
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton(projection), Collections.<String>emptyList());
assertTrue(projectionsContext.findAlias(projection.getExpression()).isPresent());
}
@Test
public void assertFindProjectionIndex() {
Projection projection = getColumnProjection();
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton(projection));
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton(projection), Collections.<String>emptyList());
Optional<Integer> actual = projectionsContext.findProjectionIndex(projection.getExpression());
assertTrue(actual.isPresent());
assertThat(actual.get(), is(1));
......@@ -93,7 +86,7 @@ public final class ProjectionsContextTest {
@Test
public void assertFindProjectionIndexFailure() {
Projection projection = getColumnProjection();
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton(projection));
ProjectionsContext projectionsContext = new ProjectionsContext(0, 0, true, Collections.singleton(projection), Collections.<String>emptyList());
Optional<Integer> actual = projectionsContext.findProjectionIndex("");
assertFalse(actual.isPresent());
}
......@@ -101,7 +94,7 @@ public final class ProjectionsContextTest {
@Test
public void assertGetAggregationProjections() {
Projection projection = getAggregationProjection();
List<AggregationProjection> items = new ProjectionsContext(0, 0, true, Arrays.asList(projection, getColumnProjection())).getAggregationProjections();
List<AggregationProjection> items = new ProjectionsContext(0, 0, true, Arrays.asList(projection, getColumnProjection()), Collections.<String>emptyList()).getAggregationProjections();
assertTrue(items.contains(projection));
assertThat(items.size(), is(1));
}
......@@ -109,97 +102,16 @@ public final class ProjectionsContextTest {
@Test
public void assertGetAggregationDistinctProjections() {
Projection projection = getAggregationDistinctProjection();
List<AggregationDistinctProjection> items = new ProjectionsContext(0, 0, true, Arrays.asList(projection, getColumnProjection())).getAggregationDistinctProjections();
List<AggregationDistinctProjection> items = new ProjectionsContext(
0, 0, true, Arrays.asList(projection, getColumnProjection()), Collections.<String>emptyList()).getAggregationDistinctProjections();
assertTrue(items.contains(projection));
assertThat(items.size(), is(1));
}
@Test
public void assertGetColumnLabelWithShorthandProjection() {
Projection projection = getShorthandProjection();
List<String> columnLabels = new ProjectionsContext(
0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.singletonList(new TableSegment(0, 0, "table")));
assertThat(columnLabels, is(Arrays.asList("id", "name")));
}
@Test
public void assertGetColumnLabelWithShorthandProjectionWithoutOwner() {
Projection projection = getShorthandProjectionWithoutOwner();
List<String> columnLabels = new ProjectionsContext(
0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.singletonList(new TableSegment(0, 0, "table")));
assertThat(columnLabels, is(Arrays.asList("id", "name")));
}
@Test
public void assertGetColumnLabelsWithCommonProjection() {
Projection projection = getColumnProjection();
List<String> columnLabels = new ProjectionsContext(0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.<TableSegment>emptyList());
assertTrue(columnLabels.contains(projection.getColumnLabel()));
}
@Test
public void assertGetColumnLabelsWithCommonProjectionAlias() {
Projection projection = getColumnProjectionWithAlias();
List<String> columnLabels = new ProjectionsContext(0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.<TableSegment>emptyList());
assertTrue(columnLabels.contains(projection.getAlias().or("")));
}
@Test
public void assertGetColumnLabelsWithExpressionProjection() {
Projection projection = getExpressionProjection();
List<String> columnLabels = new ProjectionsContext(0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.<TableSegment>emptyList());
assertTrue(columnLabels.contains(projection.getColumnLabel()));
}
@Test
public void assertGetColumnLabelsWithExpressionProjectionAlias() {
Projection projection = getExpressionSelectProjectionWithAlias();
List<String> columnLabels = new ProjectionsContext(0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.<TableSegment>emptyList());
assertTrue(columnLabels.contains(projection.getAlias().or("")));
}
@Test
public void assertGetColumnLabelsWithDerivedProjection() {
Projection projection = getDerivedProjection();
List<String> columnLabels = new ProjectionsContext(0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.<TableSegment>emptyList());
assertTrue(columnLabels.contains(projection.getColumnLabel()));
}
@Test
public void assertGetColumnLabelsWithDerivedProjectionAlias() {
Projection projection = getDerivedProjectionWithAlias();
List<String> columnLabels = new ProjectionsContext(0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.<TableSegment>emptyList());
assertTrue(columnLabels.contains(projection.getAlias().or("")));
}
@Test
public void assertGetColumnLabelsWithAggregationProjection() {
Projection projection = getAggregationProjection();
List<String> columnLabels = new ProjectionsContext(0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.<TableSegment>emptyList());
assertTrue(columnLabels.contains(projection.getColumnLabel()));
}
@Test
public void assertGetColumnLabelsWithAggregationDistinctProjection() {
Projection projection = getAggregationDistinctProjection();
List<String> columnLabels = new ProjectionsContext(0, 0, true, Collections.singletonList(projection)).getColumnLabels(createRelationMetas(), Collections.<TableSegment>emptyList());
assertTrue(columnLabels.contains(projection.getColumnLabel()));
}
private RelationMetas createRelationMetas() {
Map<String, RelationMetaData> relations = new HashMap<>(1, 1);
relations.put("table", new RelationMetaData(Arrays.asList("id", "name")));
return new RelationMetas(relations);
}
private ShorthandProjection getShorthandProjection() {
return new ShorthandProjection("table");
}
private ShorthandProjection getShorthandProjectionWithoutOwner() {
return new ShorthandProjection(null);
}
private ColumnProjection getColumnProjection() {
return new ColumnProjection("table", "name", null);
}
......@@ -208,22 +120,6 @@ public final class ProjectionsContextTest {
return new ColumnProjection("table", "name", "n");
}
private ExpressionProjection getExpressionProjection() {
return new ExpressionProjection("table.name", null);
}
private ExpressionProjection getExpressionSelectProjectionWithAlias() {
return new ExpressionProjection("table.name", "n");
}
private DerivedProjection getDerivedProjection() {
return new DerivedProjection("table.name", null);
}
private DerivedProjection getDerivedProjectionWithAlias() {
return new DerivedProjection("table.name", "n");
}
private AggregationProjection getAggregationProjection() {
return new AggregationProjection(AggregationType.COUNT, "(column)", "c");
}
......
......@@ -163,6 +163,7 @@ public final class ProjectionsContextEngineTest {
shorthandSelectItemSegment.setOwner(owner);
when(selectStatement.findSQLSegments(TableAvailable.class)).thenReturn(Collections.singletonList((TableAvailable) tableSegment));
ColumnSegment columnSegment = mock(ColumnSegment.class);
when(columnSegment.getName()).thenReturn("name");
when(columnSegment.getOwner()).thenReturn(Optional.of(tableSegment));
ColumnSelectItemSegment columnSelectItemSegment = new ColumnSelectItemSegment("ColumnSelectItemSegment", columnSegment);
columnSelectItemSegment.setOwner(owner);
......
......@@ -19,7 +19,6 @@ package org.apache.shardingsphere.sql.parser.relation.statement.impl;
import com.google.common.collect.Lists;
import org.apache.shardingsphere.sql.parser.core.constant.OrderDirection;
import org.apache.shardingsphere.sql.parser.relation.metadata.RelationMetas;
import org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext;
import org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByItem;
......@@ -37,7 +36,6 @@ import org.apache.shardingsphere.sql.parser.sql.segment.dml.order.item.OrderByIt
import org.apache.shardingsphere.sql.parser.sql.segment.generic.TableSegment;
import org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement;
import org.junit.Test;
import org.mockito.ArgumentMatchers;
import java.util.Arrays;
import java.util.Collections;
......@@ -48,7 +46,6 @@ import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
......@@ -141,22 +138,6 @@ public final class SelectSQLStatementContextTest {
assertThat(selectSQLStatementContext.getOrderByContext().getItems().iterator().next().getIndex(), is(3));
}
@Test
public void assertGetColumnLabelsWhenResultIsEmpty() {
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(
new SelectStatement(), new GroupByContext(Collections.<OrderByItem>emptyList(), 0), createOrderBy(COLUMN_ORDER_BY_WITHOUT_OWNER_ALIAS), mock(ProjectionsContext.class), null);
assertTrue(selectSQLStatementContext.getColumnLabels(mock(RelationMetas.class)).isEmpty());
}
@Test
public void assertGetColumnLabelsWhenResultIsNotEmpty() {
ProjectionsContext projectionsContext = mock(ProjectionsContext.class);
when(projectionsContext.getColumnLabels((RelationMetas) any(), ArgumentMatchers.<TableSegment>anyCollection())).thenReturn(Collections.singletonList("result"));
SelectSQLStatementContext selectSQLStatementContext = new SelectSQLStatementContext(
new SelectStatement(), new GroupByContext(Collections.<OrderByItem>emptyList(), 0), createOrderBy(COLUMN_ORDER_BY_WITHOUT_OWNER_ALIAS), projectionsContext, null);
assertFalse(selectSQLStatementContext.getColumnLabels(mock(RelationMetas.class)).isEmpty());
}
private OrderByContext createOrderBy(final String type) {
OrderByItemSegment orderByItemSegment = createOrderByItemSegment(type);
OrderByItem orderByItem = new OrderByItem(orderByItemSegment);
......@@ -179,7 +160,8 @@ public final class SelectSQLStatementContextTest {
}
private ProjectionsContext createProjectionsContext() {
return new ProjectionsContext(0, 0, true, Arrays.asList(getColumnSelectItemWithoutOwner(), getColumnSelectItemWithoutOwner(true), getColumnSelectItemWithoutOwner(false)));
return new ProjectionsContext(
0, 0, true, Arrays.asList(getColumnSelectItemWithoutOwner(), getColumnSelectItemWithoutOwner(true), getColumnSelectItemWithoutOwner(false)), Collections.<String>emptyList());
}
private Projection getColumnSelectItemWithoutOwner() {
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册