http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java new file mode 100644 index 0000000..00ff500 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java @@ -0,0 +1,290 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep; + +import org.apache.ignite.*; +import org.h2.engine.*; +import org.h2.index.*; +import org.h2.message.*; +import org.h2.result.*; +import org.h2.table.*; +import org.jetbrains.annotations.*; + +import java.util.*; +import java.util.concurrent.atomic.*; + +/** + * Merge index. + */ +public abstract class GridMergeIndex extends BaseIndex { + /** */ + protected final GridResultPage<?> END = new GridResultPage<Object>(null, null); + + /** */ + private static final int MAX_FETCH_SIZE = 100000; + + /** */ + private final AtomicInteger cnt = new AtomicInteger(0); + + /** Result sources. */ + private final AtomicInteger srcs = new AtomicInteger(0); + + /** + * Will be r/w from query execution thread only, does not need to be threadsafe. + */ + private ArrayList<Row> fetched = new ArrayList<>(); + + /** + * @param tbl Table. + * @param name Index name. + * @param type Type. + * @param cols Columns. + */ + public GridMergeIndex(GridMergeTable tbl, String name, IndexType type, IndexColumn[] cols) { + initBaseIndex(tbl, 0, name, cols, type); + } + + /** {@inheritDoc} */ + @Override public long getRowCount(Session session) { + return cnt.get(); + } + + /** {@inheritDoc} */ + @Override public long getRowCountApproximation() { + return getRowCount(null); + } + + /** + * @param srcs Number of sources. + */ + public void setNumberOfSources(int srcs) { + this.srcs.set(srcs); + } + + /** + * @param cnt Count. + */ + public void addCount(int cnt) { + this.cnt.addAndGet(cnt); + } + + /** + * @param page Page. + */ + public final void addPage(GridResultPage<?> page) { + if (!page.response().rows().isEmpty()) + addPage0(page); + else + assert page.response().isLast(); + + if (page.response().isLast()) { + int srcs0 = srcs.decrementAndGet(); + + assert srcs0 >= 0; + + if (srcs0 == 0) + addPage0(END); // We've fetched all. + } + } + + /** + * @param page Page. + */ + protected abstract void addPage0(GridResultPage<?> page); + + /** {@inheritDoc} */ + @Override public Cursor find(Session session, SearchRow first, SearchRow last) { + if (fetched == null) + throw new IgniteException("Fetched result set was too large."); + + if (fetched.size() == cnt.get()) // We've fetched all the rows. + return findAllFetched(fetched, first, last); + + return findInStream(first, last); + } + + /** + * @param first First row. + * @param last Last row. + * @return Cursor. Usually it must be {@link FetchingCursor} instance. + */ + protected abstract Cursor findInStream(@Nullable SearchRow first, @Nullable SearchRow last); + + /** + * @param fetched Fetched rows. + * @param first First row. + * @param last Last row. + * @return Cursor. + */ + protected Cursor findAllFetched(List<Row> fetched, @Nullable SearchRow first, @Nullable SearchRow last) { + return new IteratorCursor(fetched.iterator()); + } + + /** {@inheritDoc} */ + @Override public void checkRename() { + throw DbException.getUnsupportedException("rename"); + } + + /** {@inheritDoc} */ + @Override public void close(Session session) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void add(Session session, Row row) { + throw DbException.getUnsupportedException("add"); + } + + /** {@inheritDoc} */ + @Override public void remove(Session session, Row row) { + throw DbException.getUnsupportedException("remove row"); + } + + /** {@inheritDoc} */ + @Override public double getCost(Session session, int[] masks, TableFilter filter, SortOrder sortOrder) { + return getRowCountApproximation() + Constants.COST_ROW_OFFSET; + } + + /** {@inheritDoc} */ + @Override public void remove(Session session) { + throw DbException.getUnsupportedException("remove index"); + } + + /** {@inheritDoc} */ + @Override public void truncate(Session session) { + throw DbException.getUnsupportedException("truncate"); + } + + /** {@inheritDoc} */ + @Override public boolean canGetFirstOrLast() { + return false; + } + + /** {@inheritDoc} */ + @Override public Cursor findFirstOrLast(Session session, boolean first) { + throw DbException.getUnsupportedException("findFirstOrLast"); + } + + /** {@inheritDoc} */ + @Override public boolean needRebuild() { + return false; + } + + /** {@inheritDoc} */ + @Override public long getDiskSpaceUsed() { + return 0; + } + + /** + * Cursor over iterator. + */ + protected class IteratorCursor implements Cursor { + /** */ + protected Iterator<Row> iter; + + /** */ + protected Row cur; + + /** + * @param iter Iterator. + */ + public IteratorCursor(Iterator<Row> iter) { + assert iter != null; + + this.iter = iter; + } + + /** {@inheritDoc} */ + @Override public Row get() { + return cur; + } + + /** {@inheritDoc} */ + @Override public SearchRow getSearchRow() { + return get(); + } + + /** {@inheritDoc} */ + @Override public boolean next() { + cur = iter.hasNext() ? iter.next() : null; + + return cur != null; + } + + /** {@inheritDoc} */ + @Override public boolean previous() { + throw DbException.getUnsupportedException("previous"); + } + } + + /** + * Fetching cursor. + */ + protected class FetchingCursor extends IteratorCursor { + /** */ + private Iterator<Row> stream; + + /** + */ + public FetchingCursor(Iterator<Row> stream) { + super(new FetchedIterator()); + + assert stream != null; + + this.stream = stream; + } + + /** {@inheritDoc} */ + @Override public boolean next() { + if (super.next()) { + assert cur != null; + + if (iter == stream && fetched != null) { // Cache fetched rows for reuse. + if (fetched.size() == MAX_FETCH_SIZE) + fetched = null; // Throw away fetched result if it is too large. + else + fetched.add(cur); + } + + return true; + } + + if (iter == stream) // We've fetched the stream. + return false; + + iter = stream; // Switch from cached to stream. + + return next(); + } + } + + /** + * List iterator without {@link ConcurrentModificationException}. + */ + private class FetchedIterator implements Iterator<Row> { + /** */ + private int idx; + + /** {@inheritDoc} */ + @Override public boolean hasNext() { + return fetched != null && idx < fetched.size(); + } + + /** {@inheritDoc} */ + @Override public Row next() { + return fetched.get(idx++); + } + + /** {@inheritDoc} */ + @Override public void remove() { + throw new UnsupportedOperationException(); + } + } +}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java new file mode 100644 index 0000000..d5cbf9e --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java @@ -0,0 +1,85 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep; + +import org.apache.ignite.*; +import org.h2.index.*; +import org.h2.result.*; +import org.h2.table.*; +import org.h2.value.*; +import org.jetbrains.annotations.*; + +import java.util.*; +import java.util.concurrent.*; + +/** + * Unsorted merge index. + */ +public class GridMergeIndexUnsorted extends GridMergeIndex { + /** */ + private final BlockingQueue<GridResultPage<?>> queue = new LinkedBlockingQueue<>(); + + /** + * @param tbl Table. + * @param name Index name. + */ + public GridMergeIndexUnsorted(GridMergeTable tbl, String name) { + super(tbl, name, IndexType.createScan(false), IndexColumn.wrap(tbl.getColumns())); + } + + /** {@inheritDoc} */ + @Override public void addPage0(GridResultPage<?> page) { + queue.add(page); + } + + /** {@inheritDoc} */ + @Override protected Cursor findInStream(@Nullable SearchRow first, @Nullable SearchRow last) { + return new FetchingCursor(new Iterator<Row>() { + /** */ + Iterator<Value[]> iter = Collections.emptyIterator(); + + @Override public boolean hasNext() { + if (iter.hasNext()) + return true; + + GridResultPage<?> page; + + try { + page = queue.take(); + } + catch (InterruptedException e) { + throw new IgniteException("Query execution was interrupted.", e); + } + + if (page == END) { + assert queue.isEmpty() : "It must be the last page: " + queue; + + return false; // We are done. + } + + page.fetchNextPage(); + + iter = page.response().rows().iterator(); + + assert iter.hasNext(); + + return true; + } + + @Override public Row next() { + return new Row(iter.next(), 0); + } + + @Override public void remove() { + throw new UnsupportedOperationException(); + } + }); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeTable.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeTable.java new file mode 100644 index 0000000..f10aec0 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeTable.java @@ -0,0 +1,178 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep; + +import org.h2.api.*; +import org.h2.command.ddl.*; +import org.h2.engine.*; +import org.h2.index.*; +import org.h2.message.*; +import org.h2.result.*; +import org.h2.table.*; + +import java.util.*; + +/** + * Merge table for distributed queries. + */ +public class GridMergeTable extends TableBase { + /** */ + private final ArrayList<Index> idxs = new ArrayList<>(1); + + /** */ + private final GridMergeIndex idx; + + /** + * @param data Data. + */ + public GridMergeTable(CreateTableData data) { + super(data); + + idx = new GridMergeIndexUnsorted(this, "merge_scan"); + + idxs.add(idx); + } + + /** {@inheritDoc} */ + @Override public void lock(Session session, boolean exclusive, boolean force) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void close(Session ses) { + idx.close(ses); + } + + /** {@inheritDoc} */ + @Override public void unlock(Session s) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public Index addIndex(Session session, String indexName, int indexId, IndexColumn[] cols, + IndexType indexType, boolean create, String indexComment) { + throw DbException.getUnsupportedException("addIndex"); + } + + /** {@inheritDoc} */ + @Override public void removeRow(Session session, Row row) { + throw DbException.getUnsupportedException("removeRow"); + } + + /** {@inheritDoc} */ + @Override public void truncate(Session session) { + throw DbException.getUnsupportedException("truncate"); + } + + /** {@inheritDoc} */ + @Override public void addRow(Session session, Row row) { + throw DbException.getUnsupportedException("addRow"); + } + + /** {@inheritDoc} */ + @Override public void checkSupportAlter() { + throw DbException.getUnsupportedException("alter"); + } + + /** {@inheritDoc} */ + @Override public String getTableType() { + return EXTERNAL_TABLE_ENGINE; + } + + /** {@inheritDoc} */ + @Override public GridMergeIndex getScanIndex(Session session) { + return idx; + } + + /** {@inheritDoc} */ + @Override public Index getUniqueIndex() { + return null; // We don't have a PK. + } + + /** {@inheritDoc} */ + @Override public ArrayList<Index> getIndexes() { + return idxs; + } + + /** {@inheritDoc} */ + @Override public boolean isLockedExclusively() { + return false; + } + + /** {@inheritDoc} */ + @Override public long getMaxDataModificationId() { + return 0; + } + + /** {@inheritDoc} */ + @Override public boolean isDeterministic() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean canGetRowCount() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean canDrop() { + return true; + } + + /** {@inheritDoc} */ + @Override public long getRowCount(Session ses) { + return idx.getRowCount(ses); + } + + /** {@inheritDoc} */ + @Override public long getRowCountApproximation() { + return idx.getRowCountApproximation(); + } + + /** {@inheritDoc} */ + @Override public long getDiskSpaceUsed() { + return 0; + } + + /** {@inheritDoc} */ + @Override public void checkRename() { + throw DbException.getUnsupportedException("rename"); + } + + /** + * Engine. + */ + public static class Engine implements TableEngine { + /** */ + private static ThreadLocal<GridMergeTable> createdTbl = new ThreadLocal<>(); + + /** + * @return Created table. + */ + public static GridMergeTable getCreated() { + GridMergeTable tbl = createdTbl.get(); + + assert tbl != null; + + createdTbl.remove(); + + return tbl; + } + + /** {@inheritDoc} */ + @Override public Table createTable(CreateTableData data) { + GridMergeTable tbl = new GridMergeTable(data); + + createdTbl.set(tbl); + + return tbl; + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java new file mode 100644 index 0000000..67c464c --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java @@ -0,0 +1,242 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep; + +import org.apache.ignite.*; +import org.apache.ignite.cluster.*; +import org.apache.ignite.internal.*; +import org.apache.ignite.internal.processors.query.h2.*; +import org.apache.ignite.internal.util.future.*; +import org.apache.ignite.internal.util.typedef.*; +import org.apache.ignite.internal.util.typedef.internal.*; +import org.apache.ignite.lang.*; +import org.gridgain.grid.kernal.processors.cache.query.*; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.*; +import org.jdk8.backport.*; + +import java.sql.*; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; + +/** + * Reduce query executor. + */ +public class GridReduceQueryExecutor { + /** */ + private GridKernalContext ctx; + + /** */ + private IgniteH2Indexing h2; + + /** */ + private IgniteLogger log; + + /** */ + private final AtomicLong reqIdGen = new AtomicLong(); + + /** */ + private final ConcurrentMap<Long, QueryRun> runs = new ConcurrentHashMap8<>(); + + /** + * @param ctx Context. + * @param h2 H2 Indexing. + * @throws IgniteCheckedException If failed. + */ + public void start(final GridKernalContext ctx, IgniteH2Indexing h2) throws IgniteCheckedException { + this.ctx = ctx; + this.h2 = h2; + + log = ctx.log(GridReduceQueryExecutor.class); + + // TODO handle node failure. + + ctx.io().addUserMessageListener(GridTopic.TOPIC_QUERY, new IgniteBiPredicate<UUID, Object>() { + @Override public boolean apply(UUID nodeId, Object msg) { + assert msg != null; + + ClusterNode node = ctx.discovery().node(nodeId); + + if (msg instanceof GridNextPageResponse) + onNextPage(node, (GridNextPageResponse)msg); + else if (msg instanceof GridQueryFailResponse) + onFail(node, (GridQueryFailResponse)msg); + + return true; + } + }); + } + + private void onFail(ClusterNode node, GridQueryFailResponse msg) { + U.error(log, "Failed to execute query.", msg.error()); + } + + private void onNextPage(final ClusterNode node, GridNextPageResponse msg) { + final long qryReqId = msg.queryRequestId(); + final int qry = msg.query(); + final int pageSize = msg.rows().size(); + + QueryRun r = runs.get(qryReqId); + + GridMergeIndex idx = r.tbls.get(msg.query()).getScanIndex(null); + + if (msg.allRows() != -1) { // Only the first page contains row count. + idx.addCount(msg.allRows()); + + r.latch.countDown(); + } + + idx.addPage(new GridResultPage<UUID>(node.id(), msg) { + @Override public void fetchNextPage() { + try { + ctx.io().sendUserMessage(F.asList(node), new GridNextPageRequest(qryReqId, qry, pageSize)); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + }); + } + + /** + * @param space Space name. + * @param qry Query. + * @return Future. + */ + public IgniteFuture<GridCacheSqlResult> query(String space, GridCacheTwoStepQuery qry) { + long qryReqId = reqIdGen.incrementAndGet(); + + QueryRun r = new QueryRun(); + + r.tbls = new ArrayList<>(qry.mapQueries().size()); + + try { + r.conn = h2.connectionForSpace(space); + } + catch (IgniteCheckedException e) { + return new GridFinishedFutureEx<>(e); + } + + Collection<ClusterNode> nodes = ctx.grid().cluster().nodes(); // TODO filter nodes somehow? + + for (GridCacheSqlQuery mapQry : qry.mapQueries()) { + GridMergeTable tbl; + + try { + tbl = createTable(r.conn, mapQry); + } + catch (IgniteCheckedException e) { + return new GridFinishedFutureEx<>(e); + } + + tbl.getScanIndex(null).setNumberOfSources(nodes.size()); + + r.tbls.add(tbl); + } + + r.latch = new CountDownLatch(r.tbls.size() * nodes.size()); + + this.runs.put(qryReqId, r); + + try { + ctx.io().sendUserMessage(nodes, new GridQueryRequest(qryReqId, 1000, qry.mapQueries()), // TODO conf page size + GridTopic.TOPIC_QUERY, false, 0); + + r.latch.await(); + + GridCacheSqlQuery rdc = qry.reduceQuery(); + + final ResultSet res = h2.executeSqlQueryWithTimer(r.conn, rdc.query(), F.asList(rdc.parameters())); + + for (GridMergeTable tbl : r.tbls) + dropTable(r.conn, tbl.getName()); + + return new GridFinishedFuture(ctx, new Iter(res)); + } + catch (IgniteCheckedException | InterruptedException | SQLException e) { + U.closeQuiet(r.conn); + + return new GridFinishedFuture<>(ctx, e); + } + } + + /** + * @param conn Connection. + * @param tblName Table name. + * @throws SQLException If failed. + */ + private void dropTable(Connection conn, String tblName) throws SQLException { + try (Statement s = conn.createStatement()) { + s.execute("DROP TABLE " + tblName); + } + } + + /** + * @param conn Connection. + * @param qry Query. + * @return Table. + * @throws IgniteCheckedException If failed. + */ + private GridMergeTable createTable(Connection conn, GridCacheSqlQuery qry) throws IgniteCheckedException { + try { + try (PreparedStatement s = conn.prepareStatement( + "CREATE LOCAL TEMPORARY TABLE " + qry.alias() + + " ENGINE \"" + GridMergeTable.Engine.class.getName() + "\" " + + " AS SELECT * FROM (" + qry.query() + ") WHERE FALSE")) { + h2.bindParameters(s, F.asList(qry.parameters())); + + s.execute(); + } + + return GridMergeTable.Engine.getCreated(); + } + catch (SQLException e) { + U.closeQuiet(conn); + + throw new IgniteCheckedException(e); + } + } + + /** + * + */ + private static class QueryRun { + /** */ + private List<GridMergeTable> tbls; + + /** */ + private CountDownLatch latch; + + /** */ + private Connection conn; + } + + /** + * + */ + private static class Iter extends GridH2ResultSetIterator<List<?>> implements GridCacheSqlResult { + /** + * @param data Data array. + * @throws IgniteCheckedException If failed. + */ + protected Iter(ResultSet data) throws IgniteCheckedException { + super(data); + } + + /** {@inheritDoc} */ + @Override protected List<?> createRow() { + ArrayList<Object> res = new ArrayList<>(row.length); + + Collections.addAll(res, row); + + return res; + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java new file mode 100644 index 0000000..22a5dd9 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java @@ -0,0 +1,59 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep; + +import org.apache.ignite.internal.util.typedef.internal.*; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.*; + +/** + * Page result. + */ +public class GridResultPage<Z> { + /** */ + private final Z src; + + /** */ + private final GridNextPageResponse res; + + /** + * @param src Source. + * @param res Response. + */ + protected GridResultPage(Z src, GridNextPageResponse res) { + this.src = src; + this.res = res; + } + + /** + * @return Result source. + */ + public Z source() { + return src; + } + + /** + * @return Response. + */ + public GridNextPageResponse response() { + return res; + } + + /** + * Request next page. + */ + public void fetchNextPage() { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridResultPage.class, this); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageRequest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageRequest.java new file mode 100644 index 0000000..49592ca --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageRequest.java @@ -0,0 +1,59 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep.messages; + + +import java.io.*; + +/** + * Request to fetch next page. + */ +public class GridNextPageRequest implements Serializable { + /** */ + private long qryReqId; + + /** */ + private int qry; + + /** */ + private int pageSize; + + /** + * @param qryReqId Query request ID. + * @param qry Query. + * @param pageSize Page size. + */ + public GridNextPageRequest(long qryReqId, int qry, int pageSize) { + this.qryReqId = qryReqId; + this.qry = qry; + this.pageSize = pageSize; + } + + /** + * @return Query request ID. + */ + public long queryRequestId() { + return qryReqId; + } + + /** + * @return Query. + */ + public int query() { + return qry; + } + + /** + * @return Page size. + */ + public int pageSize() { + return pageSize; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageResponse.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageResponse.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageResponse.java new file mode 100644 index 0000000..fce08e1 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageResponse.java @@ -0,0 +1,180 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep.messages; + +import org.apache.ignite.internal.util.typedef.internal.*; +import org.h2.store.*; +import org.h2.value.*; + +import java.io.*; +import java.util.*; + +/** + * Next page response. + */ +public class GridNextPageResponse implements Externalizable { + /** */ + private long qryReqId; + + /** */ + private int qry; + + /** */ + private int page; + + /** */ + private int allRows; + + /** */ + private Collection<Value[]> rows; + + /** */ + private boolean last; + + /** + * For {@link Externalizable}. + */ + public GridNextPageResponse() { + // No-op. + } + + /** + * @param qryReqId Query request ID. + * @param qry Query. + * @param page Page. + * @param allRows All rows count. + * @param last Last row. + * @param rows Rows. + */ + public GridNextPageResponse(long qryReqId, int qry, int page, int allRows, boolean last, Collection<Value[]> rows) { + assert rows != null; + + this.qryReqId = qryReqId; + this.qry = qry; + this.page = page; + this.allRows = allRows; + this.last = last; + this.rows = rows; + } + + /** + * @return Query request ID. + */ + public long queryRequestId() { + return qryReqId; + } + + /** + * @return Query. + */ + public int query() { + return qry; + } + + /** + * @return Page. + */ + public int page() { + return page; + } + + /** + * @return All rows. + */ + public int allRows() { + return allRows; + } + + /** + * @return {@code true} If this is the last page. + */ + public boolean isLast() { + return last; + } + + /** + * @return Rows. + */ + public Collection<Value[]> rows() { + return rows; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeLong(qryReqId); + out.writeInt(qry); + out.writeInt(page); + out.writeBoolean(last); + out.writeInt(allRows); + + out.writeInt(rows.size()); + + if (rows.isEmpty()) + return; + + Data data = Data.create(null, 512); + + boolean first = true; + + for (Value[] row : rows) { + if (first) { + out.writeInt(row.length); + + first = false; + } + + for (Value val : row) + data.writeValue(val); + } + + out.writeInt(data.length()); + out.write(data.getBytes(), 0, data.length()); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + qryReqId = in.readLong(); + qry = in.readInt(); + page = in.readInt(); + last = in.readBoolean(); + allRows = in.readInt(); + + int rowCnt = in.readInt(); + + if (rowCnt == 0) + rows = Collections.emptyList(); + else { + rows = new ArrayList<>(rowCnt); + + int cols = in.readInt(); + int dataSize = in.readInt(); + + byte[] dataBytes = new byte[dataSize]; + + in.readFully(dataBytes); + + Data data = Data.create(null, dataBytes); + + for (int r = 0; r < rowCnt; r++) { + Value[] row = new Value[cols]; + + for (int c = 0; c < cols; c++) + row[c] = data.readValue(); + + rows.add(row); + } + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNextPageResponse.class, this); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryAck.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryAck.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryAck.java new file mode 100644 index 0000000..8d9fc89 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryAck.java @@ -0,0 +1,34 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep.messages; + +import java.io.*; + +/** + * TODO write doc + */ +public class GridQueryAck implements Serializable { + /** */ + private long reqId; + + /** + * @param reqId Request ID. + */ + public GridQueryAck(long reqId) { + this.reqId = reqId; + } + + /** + * @return Request ID. + */ + public long requestId() { + return reqId; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java new file mode 100644 index 0000000..a0eb9cd --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java @@ -0,0 +1,46 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep.messages; + +import java.io.*; + +/** + * Error message. + */ +public class GridQueryFailResponse implements Serializable { + /** */ + private long qryReqId; + + /** */ + private Throwable err; + + /** + * @param qryReqId Query request ID. + * @param err Error. + */ + public GridQueryFailResponse(long qryReqId, Throwable err) { + this.qryReqId = qryReqId; + this.err = err; + } + + /** + * @return Query request ID. + */ + public long queryRequestId() { + return qryReqId; + } + + /** + * @return Error. + */ + public Throwable error() { + return err; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java new file mode 100644 index 0000000..c37158a --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java @@ -0,0 +1,61 @@ +/* @java.file.header */ + +/* _________ _____ __________________ _____ + * __ ____/___________(_)______ /__ ____/______ ____(_)_______ + * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ + * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / + * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ + */ + +package org.apache.ignite.internal.processors.query.h2.twostep.messages; + +import org.gridgain.grid.kernal.processors.cache.query.*; + +import java.io.*; +import java.util.*; + +/** + * Query request. + */ +public class GridQueryRequest implements Serializable { + /** */ + private long reqId; + + /** */ + private int pageSize; + + /** */ + private Collection<GridCacheSqlQuery> qrys; + + /** + * @param reqId Request ID. + * @param pageSize Page size. + * @param qrys Queries. + */ + public GridQueryRequest(long reqId, int pageSize, Collection<GridCacheSqlQuery> qrys) { + this.reqId = reqId; + this.pageSize = pageSize; + this.qrys = qrys; + } + + /** + * @return Request ID. + */ + public long requestId() { + return reqId; + } + + /** + * @return Page size. + */ + public int pageSize() { + return pageSize; + } + + /** + * @return Queries. + */ + public Collection<GridCacheSqlQuery> queries() { + return qrys; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAggregateFunction.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAggregateFunction.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAggregateFunction.java deleted file mode 100644 index c9aeee9..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAggregateFunction.java +++ /dev/null @@ -1,76 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -import org.h2.util.*; - -import static org.gridgain.grid.kernal.processors.query.h2.sql.GridSqlFunctionType.*; - -/** - * Aggregate function. - */ -public class GridSqlAggregateFunction extends GridSqlFunction { - /** */ - private static final GridSqlFunctionType[] TYPE_INDEX = new GridSqlFunctionType[]{ - COUNT_ALL, COUNT, GROUP_CONCAT, SUM, MIN, MAX, AVG, -// STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP, BOOL_OR, BOOL_AND, SELECTIVITY, HISTOGRAM, - }; - - /** */ - private final boolean distinct; - - /** - * @param distinct Distinct. - * @param type Type. - */ - public GridSqlAggregateFunction(boolean distinct, GridSqlFunctionType type) { - super(type); - - this.distinct = distinct; - } - - /** - * @param distinct Distinct. - * @param typeId Type. - */ - public GridSqlAggregateFunction(boolean distinct, int typeId) { - this(distinct, TYPE_INDEX[typeId]); - } - - /** - * @return Distinct. - */ - public boolean distinct() { - return distinct; - } - - /** {@inheritDoc} */ - @Override public String getSQL() { - String text; - - switch (type) { - case GROUP_CONCAT: - throw new UnsupportedOperationException(); - - case COUNT_ALL: - return "COUNT(*)"; - - default: - text = type.name(); - - break; - } - - if (distinct) - return text + "(DISTINCT " + child().getSQL() + ")"; - - return text + StringUtils.enclose(child().getSQL()); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAlias.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAlias.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAlias.java deleted file mode 100644 index 5b00fb1..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAlias.java +++ /dev/null @@ -1,55 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -import org.h2.command.*; - -/** - * Alias for column or table. - */ -public class GridSqlAlias extends GridSqlElement { - /** */ - private final String alias; - - /** */ - private final boolean useAs; - - /** - * @param alias Alias. - * @param expr Expr. - */ - public GridSqlAlias(String alias, GridSqlElement expr) { - this(alias, expr, false); - } - - /** - * @param alias Alias. - * @param expr Expr. - * @param useAs Use 'AS' keyword. - */ - public GridSqlAlias(String alias, GridSqlElement expr, boolean useAs) { - addChild(expr); - - this.useAs = useAs; - this.alias = alias; - } - - /** {@inheritDoc} */ - @Override public String getSQL() { - return child().getSQL() + (useAs ? " AS " : " ") + Parser.quoteIdentifier(alias); - } - - /** - * @return Alias. - */ - public String alias() { - return alias; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlColumn.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlColumn.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlColumn.java deleted file mode 100644 index 460ce1c..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlColumn.java +++ /dev/null @@ -1,57 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -/** - * Column. - */ -public class GridSqlColumn extends GridSqlElement implements GridSqlValue { - /** */ - private final GridSqlElement expressionInFrom; - - /** */ - private final String colName; - - /** SQL from original query. May be qualified or unqualified column name. */ - private final String sqlText; - - /** - * @param from From. - * @param name Name. - * @param sqlText Text. - */ - public GridSqlColumn(GridSqlElement from, String name, String sqlText) { - assert sqlText != null; - - expressionInFrom = from; - colName = name; - this.sqlText = sqlText; - } - - - /** - * @return Column name. - */ - public String columnName() { - return colName; - } - - /** {@inheritDoc} */ - @Override public String getSQL() { - return sqlText; - } - - /** - * @return Expression in from. - */ - public GridSqlElement expressionInFrom() { - return expressionInFrom; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlConst.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlConst.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlConst.java deleted file mode 100644 index 01eff8c..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlConst.java +++ /dev/null @@ -1,39 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -import org.h2.value.*; - -/** - * Constant value. - */ -public class GridSqlConst extends GridSqlElement implements GridSqlValue { - /** */ - private final Value val; - - /** - * @param val Value. - */ - public GridSqlConst(Value val) { - this.val = val; - } - - /** - * @return Value. - */ - public Value value() { - return val; - } - - /** {@inheritDoc} */ - @Override public String getSQL() { - return val.getSQL(); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlElement.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlElement.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlElement.java deleted file mode 100644 index d3463fd..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlElement.java +++ /dev/null @@ -1,73 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -import java.util.*; - -/** - * Abstract SQL element. - */ -public abstract class GridSqlElement implements Cloneable { - /** */ - protected List<GridSqlElement> children = new ArrayList<>(); - - /** {@inheritDoc} */ - public abstract String getSQL(); - - /** - * @return Children. - */ - public List<GridSqlElement> children() { - return children; - } - - /** - * @param expr Expr. - * @return {@code this}. - */ - public GridSqlElement addChild(GridSqlElement expr) { - if (expr == null) - throw new NullPointerException(); - - children.add(expr); - - return this; - } - - /** - * @return First child. - */ - public GridSqlElement child() { - return children.get(0); - } - - /** - * @param idx Index. - * @return Child. - */ - public GridSqlElement child(int idx) { - return children.get(idx); - } - - /** {@inheritDoc} */ - @SuppressWarnings({"CloneCallsConstructors", "CloneDoesntDeclareCloneNotSupportedException"}) - @Override protected GridSqlElement clone() { - try { - GridSqlElement res = (GridSqlElement)super.clone(); - - res.children = new ArrayList<>(children); - - return res; - } - catch (CloneNotSupportedException e) { - throw new IllegalStateException(e); - } - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunction.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunction.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunction.java deleted file mode 100644 index fff0090..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunction.java +++ /dev/null @@ -1,142 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -import org.apache.ignite.internal.util.typedef.*; -import org.h2.util.*; -import org.h2.value.*; - -import java.util.*; - -import static org.gridgain.grid.kernal.processors.query.h2.sql.GridSqlFunctionType.*; - -/** - * Function. - */ -public class GridSqlFunction extends GridSqlElement { - /** */ - private static final Map<String, GridSqlFunctionType> TYPE_MAP = new HashMap<>(); - - /** - * - */ - static { - for (GridSqlFunctionType type : GridSqlFunctionType.values()) - TYPE_MAP.put(type.name(), type); - } - - /** */ - private final String name; - - /** */ - protected final GridSqlFunctionType type; - - /** */ - private String castType; - - /** - * @param type Function type. - */ - public GridSqlFunction(GridSqlFunctionType type) { - this(type, type.functionName()); - } - - /** - * @param type Type. - * @param name Name. - */ - private GridSqlFunction(GridSqlFunctionType type, String name) { - if (name == null) - throw new NullPointerException(); - - if (type == null) - type = UNKNOWN_FUNCTION; - - this.name = name; - this.type = type; - } - - /** - * @param name Name. - */ - public GridSqlFunction(String name) { - this(TYPE_MAP.get(name), name); - } - - /** - * @param castType Type for {@link GridSqlFunctionType#CAST} function. - * @return {@code this}. - */ - public GridSqlFunction setCastType(String castType) { - this.castType = castType; - - return this; - } - - /** {@inheritDoc} */ - @Override public String getSQL() { - StatementBuilder buff = new StatementBuilder(name); - - if (type == CASE) { - if (!children.isEmpty()) - buff.append(" ").append(child().getSQL()); - - for (int i = 1, len = children.size() - 1; i < len; i += 2) { - buff.append(" WHEN ").append(child(i).getSQL()); - buff.append(" THEN ").append(child(i + 1).getSQL()); - } - if (children.size() % 2 == 0) - buff.append(" ELSE ").append(child(children.size() - 1).getSQL()); - - return buff.append(" END").toString(); - } - - buff.append('('); - - if (type == CAST) { - assert !F.isEmpty(castType) : castType; - assert children().size() == 1; - - buff.append(child().getSQL()).append(" AS ").append(castType); - } - else if (type == CONVERT) { - assert !F.isEmpty(castType) : castType; - assert children().size() == 1; - - buff.append(child().getSQL()).append(',').append(castType); - } - else if (type == GridSqlFunctionType.EXTRACT) { - ValueString v = (ValueString)((GridSqlConst)child(0)).value(); - buff.append(v.getString()).append(" FROM ").append(child(1).getSQL()); - } - else { - for (GridSqlElement e : children) { - buff.appendExceptFirst(", "); - buff.append(e.getSQL()); - } - } - - return buff.append(')').toString(); - } - - /** - * @return Name. - */ - public String name() { - return name; - } - - /** - * @return Type. - */ - public GridSqlFunctionType type() { - return type; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunctionType.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunctionType.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunctionType.java deleted file mode 100644 index be4d90d..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunctionType.java +++ /dev/null @@ -1,78 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -import org.h2.expression.*; - -/** - * Full list of available functions see at {@link Function} - */ -public enum GridSqlFunctionType { - // Aggregate functions. - /** */ - COUNT_ALL("COUNT(*)"), - - /** */ - COUNT, - - /** */ - SUM, - - /** */ - MIN, - - /** */ - MAX, - - /** */ - AVG, - - /** */ - GROUP_CONCAT, - - // Functions with special handling. - /** */ - CASE, - - /** */ - CAST, - - /** */ - CONVERT, - - /** */ - EXTRACT, - - /** Constant for all other functions. */ - UNKNOWN_FUNCTION; - - /** */ - private final String name; - - /** - */ - GridSqlFunctionType() { - name = name(); - } - - /** - * @param name Name. - */ - GridSqlFunctionType(String name) { - this.name = name; - } - - /** - * @return Function name. - */ - public String functionName() { - return name; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlJoin.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlJoin.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlJoin.java deleted file mode 100644 index a05f973..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlJoin.java +++ /dev/null @@ -1,68 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -import org.h2.util.*; -import org.jetbrains.annotations.*; - -/** - * Join of two tables or subqueries. - */ -public class GridSqlJoin extends GridSqlElement { - /** - * @param leftTbl Left table. - * @param rightTbl Right table. - */ - public GridSqlJoin(GridSqlElement leftTbl, GridSqlElement rightTbl) { - addChild(leftTbl); - addChild(rightTbl); - } - - /** - * @return Table 1. - */ - public GridSqlElement leftTable() { - return child(0); - } - - /** - * @return Table 2. - */ - public GridSqlElement rightTable() { - return child(1); - } - - /** - * @return {@code ON} Condition. - */ - @Nullable public GridSqlElement on() { - return child(2); - } - - /** - * @return {@code true} If it is a {@code LEFT JOIN}. - */ - public boolean leftJoin() { - return false; // TODO - } - - /** {@inheritDoc} */ - @Override public String getSQL() { - StatementBuilder buff = new StatementBuilder(); - - buff.append(leftTable().getSQL()); - - buff.append(leftJoin() ? " \n LEFT JOIN " : " \n INNER JOIN "); - - buff.append(rightTable().getSQL()); - - return buff.toString(); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperation.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperation.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperation.java deleted file mode 100644 index 4477f0c..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperation.java +++ /dev/null @@ -1,73 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -/** - * Unary or binary operation. - */ -public class GridSqlOperation extends GridSqlElement { - /** */ - private final GridSqlOperationType opType; - - /** - * @param opType Operation type. - */ - public GridSqlOperation(GridSqlOperationType opType) { - this.opType = opType; - } - - /** - * @param opType Op type. - * @param arg argument. - */ - public GridSqlOperation(GridSqlOperationType opType, GridSqlElement arg) { - this(opType); - - addChild(arg); - } - - /** - * @param opType Op type. - * @param left Left. - * @param right Right. - */ - public GridSqlOperation(GridSqlOperationType opType, GridSqlElement left, GridSqlElement right) { - this(opType); - - addChild(left); - addChild(right); - } - - /** - * @return Left. - */ - public GridSqlElement left() { - return child(0); - } - - /** - * @return Right. - */ - public GridSqlElement right() { - return child(1); - } - - /** - * @return Operation type. - */ - public GridSqlOperationType opType() { - return opType; - } - - /** {@inheritDoc} */ - @Override public String getSQL() { - return opType.toSql(this); - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperationType.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperationType.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperationType.java deleted file mode 100644 index 35fe568..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperationType.java +++ /dev/null @@ -1,202 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -import org.h2.util.*; - -/** - * Operation type. - */ -public enum GridSqlOperationType { - // from org.h2.expression.Operation - CONCAT(2, new BiExpressionSqlGenerator("||")), - PLUS(2, new BiExpressionSqlGenerator("+")), - MINUS(2, new BiExpressionSqlGenerator("-")), - MULTIPLY(2, new BiExpressionSqlGenerator("*")), - DIVIDE(2, new BiExpressionSqlGenerator("/")), - MODULUS(2, new BiExpressionSqlGenerator("%")), - NEGATE(1, new PrefixSqlGenerator("-")), - - // from org.h2.expression.Comparison - EQUAL(2, new BiExpressionSqlGenerator("=")), - EQUAL_NULL_SAFE(2, new BiExpressionSqlGenerator("IS")), - BIGGER_EQUAL(2, new BiExpressionSqlGenerator(">=")), - BIGGER(2, new BiExpressionSqlGenerator(">")), - SMALLER_EQUAL(2, new BiExpressionSqlGenerator("<=")), - SMALLER(2, new BiExpressionSqlGenerator("<")), - NOT_EQUAL(2, new BiExpressionSqlGenerator("<>")), - NOT_EQUAL_NULL_SAFE(2, new BiExpressionSqlGenerator("IS NOT")), - - SPATIAL_INTERSECTS(2, new IntersectsSqlGenerator()), - IS_NULL(1, new SuffixSqlGenerator("IS NULL")), - IS_NOT_NULL(1, new SuffixSqlGenerator("IS NOT NULL")), - - NOT(1, new PrefixSqlGenerator("NOT")), - - // from org.h2.expression.ConditionAndOr - AND(2, new BiExpressionSqlGenerator("AND")), - OR(2, new BiExpressionSqlGenerator("OR")), - - // from - REGEXP(2, new BiExpressionSqlGenerator("REGEXP")), - LIKE(2, new BiExpressionSqlGenerator("LIKE")), - - IN(-1, new ConditionInSqlGenerator()), - - ; - /** */ - private final SqlGenerator sqlGenerator; - - /** */ - private final int childrenCnt; - - /** - * @param childrenCnt Children count. - * @param sqlGenerator sqlGenerator. - */ - GridSqlOperationType(int childrenCnt, SqlGenerator sqlGenerator) { - this.childrenCnt = childrenCnt; - this.sqlGenerator = sqlGenerator; - } - - /** - * @param operation Operation. - */ - public String toSql(GridSqlOperation operation) { - return sqlGenerator.getSql(operation); - } - - /** - * @return Children count. - */ - public int childrenCount() { - return childrenCnt; - } - - /** - * - */ - private static interface SqlGenerator { - - /** - * @param operation Operation expression. - */ - public String getSql(GridSqlOperation operation); - } - - /** - * - */ - private static class BiExpressionSqlGenerator implements SqlGenerator { - - /** */ - private final String delim; - - /** - * @param delim Delimiter. - */ - private BiExpressionSqlGenerator(String delim) { - this.delim = delim; - } - - /** {@inheritDoc} */ - @Override public String getSql(GridSqlOperation operation) { - assert operation.opType().childrenCnt == 2; - - return '(' + operation.child(0).getSQL() + " " + delim + " " + operation.child(1).getSQL() + ')'; - } - } - - /** - * - */ - private static class IntersectsSqlGenerator implements SqlGenerator { - - /** {@inheritDoc} */ - @Override public String getSql(GridSqlOperation operation) { - assert operation.opType().childrenCnt == 2; - - return "(INTERSECTS(" + operation.child(0) + ", " + operation.child(1) + "))"; - } - } - - /** - * - */ - private static class PrefixSqlGenerator implements SqlGenerator { - /** */ - private final String text; - - /** - * @param text Text. - */ - private PrefixSqlGenerator(String text) { - this.text = text; - } - - /** {@inheritDoc} */ - @Override public String getSql(GridSqlOperation operation) { - assert operation.opType().childrenCnt == 1; - - return '(' + text + ' ' + operation.child().getSQL() + ')'; - } - } - - /** - * - */ - private static class SuffixSqlGenerator implements SqlGenerator { - /** */ - private final String text; - - /** - * @param text Text. - */ - private SuffixSqlGenerator(String text) { - this.text = text; - } - - /** {@inheritDoc} */ - @Override public String getSql(GridSqlOperation operation) { - assert operation.opType().childrenCnt == 1; - - return '(' + operation.child().getSQL() + ' ' + text + ')'; - } - } - - /** - * - */ - private static class ConditionInSqlGenerator implements SqlGenerator { - - /** {@inheritDoc} */ - @Override public String getSql(GridSqlOperation operation) { - StatementBuilder buff = new StatementBuilder("("); - - buff.append(operation.child(0).getSQL()).append(" IN("); - - assert operation.children().size() > 1; - - if (operation.children().size() == 2) { - String child = operation.child(1).getSQL(); - - buff.append(' ').append(StringUtils.unEnclose(child)).append(' '); - } - else { - for (int i = 1; i < operation.children().size(); i++) { - buff.appendExceptFirst(", "); - buff.append(operation.child(i).getSQL()); - } - } - - return buff.append("))").toString(); - } - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlParameter.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlParameter.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlParameter.java deleted file mode 100644 index 0b7da9e..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlParameter.java +++ /dev/null @@ -1,44 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -/** - * Query parameter. - */ -public class GridSqlParameter extends GridSqlElement implements GridSqlValue { - /** Index. */ - private int idx; - - /** - * @param idx Index. - */ - public GridSqlParameter(int idx) { - this.idx = idx; - } - - /** {@inheritDoc} */ - @Override public String getSQL() { - return "?" + (idx + 1); - } - - /** - * @return Index. - */ - public int index() { - return idx; - } - - /** - * @param idx New index. - */ - public void index(int idx) { - this.idx = idx; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlQueryParser.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlQueryParser.java deleted file mode 100644 index 5c7c762..0000000 --- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlQueryParser.java +++ /dev/null @@ -1,495 +0,0 @@ -/* @java.file.header */ - -/* _________ _____ __________________ _____ - * __ ____/___________(_)______ /__ ____/______ ____(_)_______ - * _ / __ __ ___/__ / _ __ / _ / __ _ __ `/__ / __ __ \ - * / /_/ / _ / _ / / /_/ / / /_/ / / /_/ / _ / _ / / / - * \____/ /_/ /_/ \_,__/ \____/ \__,_/ /_/ /_/ /_/ - */ - -package org.gridgain.grid.kernal.processors.query.h2.sql; - -import org.apache.ignite.*; -import org.h2.command.dml.*; -import org.h2.engine.*; -import org.h2.expression.*; -import org.h2.jdbc.*; -import org.h2.result.*; -import org.h2.table.*; -import org.jetbrains.annotations.*; - -import java.lang.reflect.*; -import java.sql.*; -import java.util.*; -import java.util.Set; - -import static org.gridgain.grid.kernal.processors.query.h2.sql.GridSqlOperationType.*; - -/** - * H2 Query parser. - */ -@SuppressWarnings("TypeMayBeWeakened") -public class GridSqlQueryParser { - /** */ - private static final GridSqlOperationType[] OPERATION_OP_TYPES = new GridSqlOperationType[]{CONCAT, PLUS, MINUS, MULTIPLY, DIVIDE, null, MODULUS}; - - /** */ - private static final GridSqlOperationType[] COMPARISON_TYPES = new GridSqlOperationType[]{EQUAL, BIGGER_EQUAL, BIGGER, SMALLER_EQUAL, - SMALLER, NOT_EQUAL, IS_NULL, IS_NOT_NULL, - null, null, null, SPATIAL_INTERSECTS /* 11 */, null, null, null, null, EQUAL_NULL_SAFE /* 16 */, null, null, null, null, - NOT_EQUAL_NULL_SAFE /* 21 */}; - - /** */ - private static final Getter<Select, Expression> CONDITION = getter(Select.class, "condition"); - - /** */ - private static final Getter<Select, int[]> GROUP_INDEXES = getter(Select.class, "groupIndex"); - - /** */ - private static final Getter<Operation, Integer> OPERATION_TYPE = getter(Operation.class, "opType"); - - /** */ - private static final Getter<Operation, Expression> OPERATION_LEFT = getter(Operation.class, "left"); - - /** */ - private static final Getter<Operation, Expression> OPERATION_RIGHT = getter(Operation.class, "right"); - - /** */ - private static final Getter<Comparison, Integer> COMPARISON_TYPE = getter(Comparison.class, "compareType"); - - /** */ - private static final Getter<Comparison, Expression> COMPARISON_LEFT = getter(Comparison.class, "left"); - - /** */ - private static final Getter<Comparison, Expression> COMPARISON_RIGHT = getter(Comparison.class, "right"); - - /** */ - private static final Getter<ConditionAndOr, Integer> ANDOR_TYPE = getter(ConditionAndOr.class, "andOrType"); - - /** */ - private static final Getter<ConditionAndOr, Expression> ANDOR_LEFT = getter(ConditionAndOr.class, "left"); - - /** */ - private static final Getter<ConditionAndOr, Expression> ANDOR_RIGHT = getter(ConditionAndOr.class, "right"); - - /** */ - private static final Getter<TableView, Query> VIEW_QUERY = getter(TableView.class, "viewQuery"); - - /** */ - private static final Getter<TableFilter, String> ALIAS = getter(TableFilter.class, "alias"); - - /** */ - private static final Getter<Select, Integer> HAVING_INDEX = getter(Select.class, "havingIndex"); - - /** */ - private static final Getter<ConditionIn, Expression> LEFT_CI = getter(ConditionIn.class, "left"); - - /** */ - private static final Getter<ConditionIn, List<Expression>> VALUE_LIST_CI = getter(ConditionIn.class, "valueList"); - - /** */ - private static final Getter<ConditionInConstantSet, Expression> LEFT_CICS = - getter(ConditionInConstantSet.class, "left"); - - /** */ - private static final Getter<ConditionInConstantSet, List<Expression>> VALUE_LIST_CICS = - getter(ConditionInConstantSet.class, "valueList"); - - /** */ - private static final Getter<ConditionInSelect, Expression> LEFT_CIS = getter(ConditionInSelect.class, "left"); - - /** */ - private static final Getter<ConditionInSelect, Boolean> ALL = getter(ConditionInSelect.class, "all"); - - /** */ - private static final Getter<ConditionInSelect, Integer> COMPARE_TYPE = getter(ConditionInSelect.class, - "compareType"); - - /** */ - private static final Getter<ConditionInSelect, Query> QUERY = getter(ConditionInSelect.class, "query"); - - /** */ - private static final Getter<CompareLike, Expression> LEFT = getter(CompareLike.class, "left"); - - /** */ - private static final Getter<CompareLike, Expression> RIGHT = getter(CompareLike.class, "right"); - - /** */ - private static final Getter<CompareLike, Expression> ESCAPE = getter(CompareLike.class, "escape"); - - /** */ - private static final Getter<CompareLike, Boolean> REGEXP_CL = getter(CompareLike.class, "regexp"); - - /** */ - private static final Getter<Aggregate, Boolean> DISTINCT = getter(Aggregate.class, "distinct"); - - /** */ - private static final Getter<Aggregate, Integer> TYPE = getter(Aggregate.class, "type"); - - /** */ - private static final Getter<Aggregate, Expression> ON = getter(Aggregate.class, "on"); - - /** */ - private final IdentityHashMap<Object, Object> h2ObjToGridObj = new IdentityHashMap<>(); - - /** - * @param conn Connection. - * @param select Select query. - * @return Parsed select query. - */ - public static GridSqlSelect parse(Connection conn, String select) { - Session ses = (Session)((JdbcConnection)conn).getSession(); - - return new GridSqlQueryParser().parse((Select)ses.prepare(select)); - } - - /** - * @param filter Filter. - */ - private GridSqlElement parse(TableFilter filter) { - GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(filter); - - if (res == null) { - Table tbl = filter.getTable(); - - if (tbl instanceof TableBase) - res = new GridSqlTable(tbl.getSchema().getName(), tbl.getName()); - else if (tbl instanceof TableView) { - Query qry = VIEW_QUERY.get((TableView)tbl); - - assert0(qry instanceof Select, qry); - - res = new GridSqlSubquery(parse((Select)qry)); - } - else - throw new IgniteException("Unsupported query: " + filter); - - String alias = ALIAS.get(filter); - - if (alias != null) - res = new GridSqlAlias(alias, res, false); - - h2ObjToGridObj.put(filter, res); - } - - return res; - } - - /** - * @param select Select. - */ - public GridSqlSelect parse(Select select) { - GridSqlSelect res = (GridSqlSelect)h2ObjToGridObj.get(select); - - if (res != null) - return res; - - res = new GridSqlSelect(); - - h2ObjToGridObj.put(select, res); - - res.distinct(select.isDistinct()); - - Expression where = CONDITION.get(select); - res.where(parseExpression(where)); - - Set<TableFilter> allFilers = new HashSet<>(select.getTopFilters()); - - GridSqlElement from = null; - - TableFilter filter = select.getTopTableFilter(); - do { - assert0(filter != null, select); - assert0(!filter.isJoinOuter(), select); - assert0(filter.getNestedJoin() == null, select); - assert0(filter.getJoinCondition() == null, select); - assert0(filter.getFilterCondition() == null, select); - - GridSqlElement gridFilter = parse(filter); - - from = from == null ? gridFilter : new GridSqlJoin(from, gridFilter); - - allFilers.remove(filter); - - filter = filter.getJoin(); - } - while (filter != null); - - res.from(from); - - assert allFilers.isEmpty(); - - ArrayList<Expression> expressions = select.getExpressions(); - - for (Expression exp : expressions) - res.addExpression(parseExpression(exp)); - - int[] grpIdx = GROUP_INDEXES.get(select); - - if (grpIdx != null) { - res.groupColumns(grpIdx); - - for (int idx : grpIdx) - res.addGroupExpression(parseExpression(expressions.get(idx))); - } - - assert0(select.getHaving() == null, select); - - int havingIdx = HAVING_INDEX.get(select); - - if (havingIdx >= 0) - res.having(parseExpression(expressions.get(havingIdx))); - - for (int i = 0; i < select.getColumnCount(); i++) - res.addSelectExpression(parseExpression(expressions.get(i))); - - SortOrder sortOrder = select.getSortOrder(); - - if (sortOrder != null) { - int[] indexes = sortOrder.getQueryColumnIndexes(); - int[] sortTypes = sortOrder.getSortTypes(); - - for (int i = 0; i < indexes.length; i++) - res.addSort(parseExpression(expressions.get(indexes[i])), sortTypes[i]); - } - - return res; - } - - /** - * @param expression Expression. - */ - private GridSqlElement parseExpression(@Nullable Expression expression) { - if (expression == null) - return null; - - GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(expression); - - if (res == null) { - res = parseExpression0(expression); - - h2ObjToGridObj.put(expression, res); - } - - return res; - } - - /** - * @param expression Expression. - */ - private GridSqlElement parseExpression0(Expression expression) { - if (expression instanceof ExpressionColumn) { - TableFilter tblFilter = ((ExpressionColumn)expression).getTableFilter(); - - GridSqlElement gridTblFilter = parse(tblFilter); - - return new GridSqlColumn(gridTblFilter, expression.getColumnName(), expression.getSQL()); - } - - if (expression instanceof Alias) - return new GridSqlAlias(expression.getAlias(), parseExpression(expression.getNonAliasExpression()), true); - - if (expression instanceof ValueExpression) - return new GridSqlConst(expression.getValue(null)); - - if (expression instanceof Operation) { - Operation operation = (Operation)expression; - - Integer type = OPERATION_TYPE.get(operation); - - if (type == Operation.NEGATE) { - assert OPERATION_RIGHT.get(operation) == null; - - return new GridSqlOperation(GridSqlOperationType.NEGATE, parseExpression(OPERATION_LEFT.get(operation))); - } - - return new GridSqlOperation(OPERATION_OP_TYPES[type], - parseExpression(OPERATION_LEFT.get(operation)), - parseExpression(OPERATION_RIGHT.get(operation))); - } - - if (expression instanceof Comparison) { - Comparison cmp = (Comparison)expression; - - GridSqlOperationType opType = COMPARISON_TYPES[COMPARISON_TYPE.get(cmp)]; - - assert opType != null : COMPARISON_TYPE.get(cmp); - - GridSqlElement left = parseExpression(COMPARISON_LEFT.get(cmp)); - - if (opType.childrenCount() == 1) - return new GridSqlOperation(opType, left); - - GridSqlElement right = parseExpression(COMPARISON_RIGHT.get(cmp)); - - return new GridSqlOperation(opType, left, right); - } - - if (expression instanceof ConditionNot) - return new GridSqlOperation(NOT, parseExpression(expression.getNotIfPossible(null))); - - if (expression instanceof ConditionAndOr) { - ConditionAndOr andOr = (ConditionAndOr)expression; - - int type = ANDOR_TYPE.get(andOr); - - assert type == ConditionAndOr.AND || type == ConditionAndOr.OR; - - return new GridSqlOperation(type == ConditionAndOr.AND ? AND : OR, - parseExpression(ANDOR_LEFT.get(andOr)), parseExpression(ANDOR_RIGHT.get(andOr))); - } - - if (expression instanceof Subquery) { - Query qry = ((Subquery)expression).getQuery(); - - assert0(qry instanceof Select, expression); - - return new GridSqlSubquery(parse((Select) qry)); - } - - if (expression instanceof ConditionIn) { - GridSqlOperation res = new GridSqlOperation(IN); - - res.addChild(parseExpression(LEFT_CI.get((ConditionIn) expression))); - - List<Expression> vals = VALUE_LIST_CI.get((ConditionIn)expression); - - for (Expression val : vals) - res.addChild(parseExpression(val)); - - return res; - } - - if (expression instanceof ConditionInConstantSet) { - GridSqlOperation res = new GridSqlOperation(IN); - - res.addChild(parseExpression(LEFT_CICS.get((ConditionInConstantSet) expression))); - - List<Expression> vals = VALUE_LIST_CICS.get((ConditionInConstantSet)expression); - - for (Expression val : vals) - res.addChild(parseExpression(val)); - - return res; - } - - if (expression instanceof ConditionInSelect) { - GridSqlOperation res = new GridSqlOperation(IN); - - boolean all = ALL.get((ConditionInSelect)expression); - int compareType = COMPARE_TYPE.get((ConditionInSelect)expression); - - assert0(!all, expression); - assert0(compareType == Comparison.EQUAL, expression); - - res.addChild(parseExpression(LEFT_CIS.get((ConditionInSelect) expression))); - - Query qry = QUERY.get((ConditionInSelect)expression); - - assert0(qry instanceof Select, qry); - - res.addChild(new GridSqlSubquery(parse((Select) qry))); - - return res; - } - - if (expression instanceof CompareLike) { - assert0(ESCAPE.get((CompareLike)expression) == null, expression); - - boolean regexp = REGEXP_CL.get((CompareLike)expression); - - return new GridSqlOperation(regexp ? REGEXP : LIKE, parseExpression(LEFT.get((CompareLike) expression)), - parseExpression(RIGHT.get((CompareLike) expression))); - } - - if (expression instanceof Function) { - Function f = (Function)expression; - - GridSqlFunction res = new GridSqlFunction(f.getName()); - - for (Expression arg : f.getArgs()) - res.addChild(parseExpression(arg)); - - if (f.getFunctionType() == Function.CAST || f.getFunctionType() == Function.CONVERT) - res.setCastType(new Column(null, f.getType(), f.getPrecision(), f.getScale(), f.getDisplaySize()) - .getCreateSQL()); - - return res; - } - - if (expression instanceof Parameter) - return new GridSqlParameter(((Parameter)expression).getIndex()); - - if (expression instanceof Aggregate) { - GridSqlAggregateFunction res = new GridSqlAggregateFunction(DISTINCT.get((Aggregate)expression), - TYPE.get((Aggregate)expression)); - - Expression on = ON.get((Aggregate)expression); - - if (on != null) - res.addChild(parseExpression(on)); - - return res; - } - - throw new IgniteException("Unsupported expression: " + expression + " [type=" + - expression.getClass().getSimpleName() + ']'); - } - - /** - * @param cond Condition. - * @param o Object. - */ - private static void assert0(boolean cond, Object o) { - if (!cond) - throw new IgniteException("Unsupported query: " + o); - } - - /** - * @param cls Class. - * @param fldName Fld name. - */ - private static <T, R> Getter<T, R> getter(Class<T> cls, String fldName) { - Field field; - - try { - field = cls.getDeclaredField(fldName); - } - catch (NoSuchFieldException e) { - throw new RuntimeException(e); - } - - field.setAccessible(true); - - return new Getter<>(field); - } - - /** - * Field getter. - */ - @SuppressWarnings("unchecked") - private static class Getter<T, R> { - /** */ - private final Field fld; - - /** - * @param fld Fld. - */ - private Getter(Field fld) { - this.fld = fld; - } - - /** - * @param obj Object. - * @return Result. - */ - public R get(T obj) { - try { - return (R)fld.get(obj); - } - catch (IllegalAccessException e) { - throw new IgniteException(e); - } - } - } -}