Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7BE2718B9D for ; Wed, 11 Nov 2015 21:06:25 +0000 (UTC) Received: (qmail 58851 invoked by uid 500); 11 Nov 2015 21:06:25 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 58813 invoked by uid 500); 11 Nov 2015 21:06:24 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 58804 invoked by uid 99); 11 Nov 2015 21:06:24 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 11 Nov 2015 21:06:24 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 88A3CE0418; Wed, 11 Nov 2015 21:06:24 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sergi@apache.org To: commits@ignite.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: ignite git commit: ignite-1282-sql - row factory + meta table Date: Wed, 11 Nov 2015 21:06:24 +0000 (UTC) Repository: ignite Updated Branches: refs/heads/ignite-1282-sql c1c408a6b -> 06a179242 ignite-1282-sql - row factory + meta table Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/06a17924 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/06a17924 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/06a17924 Branch: refs/heads/ignite-1282-sql Commit: 06a179242758c5657e699f2b6d6167b9d1c5355e Parents: c1c408a Author: S.Vladykin Authored: Thu Nov 12 00:06:11 2015 +0300 Committer: S.Vladykin Committed: Thu Nov 12 00:06:11 2015 +0300 ---------------------------------------------------------------------- .../processors/query/h2/IgniteH2Indexing.java | 6 +- .../query/h2/opt/GridH2AbstractKeyValueRow.java | 5 - .../query/h2/opt/GridH2DefaultTableEngine.java | 38 ++ .../query/h2/opt/GridH2MetaTable.java | 383 +++++++++++++++++++ .../processors/query/h2/opt/GridH2Row.java | 8 +- .../query/h2/opt/GridH2RowFactory.java | 52 +-- .../processors/query/h2/opt/GridH2Table.java | 2 +- .../query/h2/opt/GridLuceneIndex.java | 4 +- .../h2/twostep/GridMergeIndexUnsorted.java | 2 +- .../h2/GridIndexingSpiAbstractSelfTest.java | 5 +- .../query/h2/opt/GridH2TableSelfTest.java | 3 +- 11 files changed, 466 insertions(+), 42 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index 023967d..0f3fb61 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -80,6 +80,7 @@ import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter; import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor; import org.apache.ignite.internal.processors.query.GridQueryIndexing; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine; import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase; import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOffheap; import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap; @@ -175,7 +176,8 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** Default DB options. */ private static final String DB_OPTIONS = ";LOCK_MODE=3;MULTI_THREADED=1;DB_CLOSE_ON_EXIT=FALSE" + ";DEFAULT_LOCK_TIMEOUT=10000;FUNCTIONS_IN_SCHEMA=true;OPTIMIZE_REUSE_RESULTS=0;QUERY_CACHE_SIZE=0;" + - "RECOMPILE_ALWAYS=1;MAX_OPERATION_MEMORY=0"; + "RECOMPILE_ALWAYS=1;MAX_OPERATION_MEMORY=0;ROW_FACTORY=\"" + GridH2RowFactory.class.getName() + "\"" + + ";DEFAULT_TABLE_ENGINE=" + GridH2DefaultTableEngine.class.getName(); /** Field name for key. */ public static final String KEY_FIELD_NAME = "_key"; @@ -2231,7 +2233,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { throws IgniteCheckedException { try { if (val == null) // Only can happen for remove operation, can create simple search row. - return GridH2RowFactory.INSTANCE.create(wrap(key, keyType)); + return GridH2RowFactory.create(wrap(key, keyType)); return schema.offheap == null ? new GridH2KeyValueRowOnheap(this, key, keyType, val, valType, expirationTime) : http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java index b876bd8..3d819da 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java @@ -473,11 +473,6 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row { } /** {@inheritDoc} */ - @Override public Value[] getValueList() { - throw new UnsupportedOperationException(); - } - - /** {@inheritDoc} */ @Override public void setValue(int idx, Value v) { if (idx == VAL_COL) val = v; http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2DefaultTableEngine.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2DefaultTableEngine.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2DefaultTableEngine.java new file mode 100644 index 0000000..f53f1b3 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2DefaultTableEngine.java @@ -0,0 +1,38 @@ +/* + * 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.ignite.internal.processors.query.h2.opt; + +import org.h2.api.TableEngine; +import org.h2.command.ddl.CreateTableData; +import org.h2.table.RegularTable; +import org.h2.table.Table; + +/** + * Default table engine. + */ +public class GridH2DefaultTableEngine implements TableEngine { + /** {@inheritDoc} */ + @Override public Table createTable(CreateTableData data) { + assert !data.persistData && !data.persistIndexes; + + if (data.isHidden && data.id == 0 && "SYS".equals(data.tableName)) + return new GridH2MetaTable(data); + + return new RegularTable(data); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java new file mode 100644 index 0000000..d8f2e08 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2MetaTable.java @@ -0,0 +1,383 @@ +/* + * 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.ignite.internal.processors.query.h2.opt; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; +import org.h2.command.ddl.CreateTableData; +import org.h2.engine.Session; +import org.h2.index.BaseIndex; +import org.h2.index.Cursor; +import org.h2.index.Index; +import org.h2.index.IndexCondition; +import org.h2.index.IndexType; +import org.h2.index.SingleRowCursor; +import org.h2.message.DbException; +import org.h2.result.Row; +import org.h2.result.SearchRow; +import org.h2.result.SortOrder; +import org.h2.table.Column; +import org.h2.table.IndexColumn; +import org.h2.table.TableBase; +import org.h2.table.TableFilter; +import org.h2.value.Value; +import org.h2.value.ValueInt; +import org.jsr166.ConcurrentHashMap8; + +/** + * Meta table. + */ +public class GridH2MetaTable extends TableBase { + /** */ + private static final int ID = 0; + + /** */ + private final MetaIndex index; + + /** */ + private final AtomicLong dataModificationId = new AtomicLong(); + + /** */ + private final Set fakeExclusiveSet = Collections.newSetFromMap( + new ConcurrentHashMap8()); + + /** + * @param data Data. + */ + public GridH2MetaTable(CreateTableData data) { + super(data); + + ArrayList cols = data.columns; + assert cols.size() == 4 : cols; + + Column id = cols.get(ID); + assert "ID".equals(id.getName()) && id.getType() == Value.INT : cols; + assert id.getColumnId() == ID; + + index = new MetaIndex(); + } + + /** {@inheritDoc} */ + @Override public Row getTemplateRow() { + return new MetaRow(); + } + + /** {@inheritDoc} */ + @Override public SearchRow getTemplateSimpleRow(boolean singleColumn) { + if (singleColumn) + return GridH2RowFactory.create((Value)null); + + return new MetaRow(); + } + + /** {@inheritDoc} */ + @Override public boolean lock(Session session, boolean exclusive, boolean forceLockEvenInMvcc) { + if (fakeExclusiveSet.contains(session)) + return true; + + if (exclusive) + fakeExclusiveSet.add(session); + + return false; + } + + /** {@inheritDoc} */ + @Override public void unlock(Session s) { + fakeExclusiveSet.remove(s); + } + + /** {@inheritDoc} */ + @Override public void close(Session session) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public Index addIndex(Session session, String indexName, int indexId, + IndexColumn[] cols, IndexType indexType, boolean create, String indexComment) { + assert cols.length == 1 : "len: " + cols.length; + + int colId = cols[0].column.getColumnId(); + assert colId == ID : "colId: " + colId; + + return index; + } + + /** {@inheritDoc} */ + @Override public void removeRow(Session session, Row row) { + dataModificationId.incrementAndGet(); + index.remove(session, row); + } + + /** {@inheritDoc} */ + @Override public void truncate(Session session) { + dataModificationId.incrementAndGet(); + index.truncate(session); + } + + /** {@inheritDoc} */ + @Override public void addRow(Session session, Row row) { + dataModificationId.incrementAndGet(); + index.add(session, row); + } + + /** {@inheritDoc} */ + @Override public void checkSupportAlter() { + throw DbException.getUnsupportedException("alter"); + } + + /** {@inheritDoc} */ + @Override public String getTableType() { + return SYSTEM_TABLE; + } + + /** {@inheritDoc} */ + @Override public Index getScanIndex(Session session) { + return index; + } + + /** {@inheritDoc} */ + @Override public Index getUniqueIndex() { + return index; + } + + /** {@inheritDoc} */ + @Override public ArrayList getIndexes() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean isLockedExclusively() { + return !fakeExclusiveSet.isEmpty(); + } + + /** {@inheritDoc} */ + @Override public boolean isLockedExclusivelyBy(Session s) { + return fakeExclusiveSet.contains(s); + } + + /** {@inheritDoc} */ + @Override public long getMaxDataModificationId() { + return dataModificationId.get(); + } + + /** {@inheritDoc} */ + @Override public boolean isDeterministic() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean canGetRowCount() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean canDrop() { + return false; + } + + /** {@inheritDoc} */ + @Override public long getRowCount(Session session) { + return index.getRowCount(session); + } + + /** {@inheritDoc} */ + @Override public long getRowCountApproximation() { + return index.getRowCountApproximation(); + } + + /** {@inheritDoc} */ + @Override public long getDiskSpaceUsed() { + return 0; + } + + /** {@inheritDoc} */ + @Override public void checkRename() { + throw DbException.getUnsupportedException("rename"); + } + + /** + * Get value row. + */ + private static class MetaRow extends GridH2Row { + /** */ + private Value v0; + + /** */ + private Value v1; + + /** */ + private Value v2; + + /** */ + private Value v3; + + /** {@inheritDoc} */ + @Override public int getColumnCount() { + return 4; + } + + /** {@inheritDoc} */ + @Override public Value getValue(int idx) { + switch (idx) { + case 0: + return v0; + + case 1: + return v1; + + case 2: + return v2; + + case 3: + return v3; + + default: + throw new IllegalStateException("Index: " + idx); + } + } + + /** {@inheritDoc} */ + @Override public void setValue(int idx, Value v) { + switch (idx) { + case 0: + v0 = v; + + break; + + case 1: + v1 = v; + + break; + + case 2: + v2 = v; + + break; + + case 3: + v3 = v; + + break; + + default: + throw new IllegalStateException("Index: " + idx); + } + } + } + + /** + * Met index. + */ + private static class MetaIndex extends BaseIndex { + /** */ + private final ConcurrentMap rows = new ConcurrentHashMap8<>(); + + /** {@inheritDoc} */ + @Override public void checkRename() { + throw DbException.getUnsupportedException("rename"); + } + + /** {@inheritDoc} */ + @Override public void close(Session session) { + // No-op. + } + + /** + * @param row Row. + * @return ID. + */ + private static ValueInt id(SearchRow row) { + Value id = row.getValue(ID); + + assert id != null; + + return (ValueInt)id; + } + + /** {@inheritDoc} */ + @Override public void add(Session session, Row row) { + rows.put(id(row), (GridH2Row)row); + } + + /** {@inheritDoc} */ + @Override public void remove(Session session, Row row) { + rows.remove(id(row), row); + } + + /** {@inheritDoc} */ + @Override public Cursor find(Session session, SearchRow first, SearchRow last) { + if (first == null || last == null || !Objects.equals(id(first), id(last))) + return new GridH2Cursor(rows.values().iterator()); + + return new SingleRowCursor(rows.get(id(first))); + } + + /** {@inheritDoc} */ + @Override public double getCost(Session session, int[] masks, TableFilter[] filters, + int filter, SortOrder sortOrder) { + if ((masks[ID] & IndexCondition.EQUALITY) == IndexCondition.EQUALITY) + return 1; + + return 1000 + rows.size(); + } + + /** {@inheritDoc} */ + @Override public void remove(Session session) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void truncate(Session session) { + rows.clear(); + } + + /** {@inheritDoc} */ + @Override public boolean canGetFirstOrLast() { + return false; + } + + /** {@inheritDoc} */ + @Override public Cursor findFirstOrLast(Session session, boolean first) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean needRebuild() { + return false; + } + + /** {@inheritDoc} */ + @Override public long getRowCount(Session session) { + return rows.size(); + } + + /** {@inheritDoc} */ + @Override public long getRowCountApproximation() { + return getRowCount(null); + } + + /** {@inheritDoc} */ + @Override public long getDiskSpaceUsed() { + return 0; + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java index 0d6cc22..9486a2e 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java @@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.query.h2.opt; import org.h2.result.Row; import org.h2.result.SearchRow; import org.h2.store.Data; +import org.h2.value.Value; /** * Row with locking support needed for unique key conflicts resolution. @@ -77,7 +78,7 @@ public abstract class GridH2Row extends Row implements GridSearchRowPointer { /** {@inheritDoc} */ @Override public void commit() { - throw new UnsupportedOperationException(); + // No-op. } /** {@inheritDoc} */ @@ -109,4 +110,9 @@ public abstract class GridH2Row extends Row implements GridSearchRowPointer { @Override public int getMemory() { throw new UnsupportedOperationException(); } + + /** {@inheritDoc} */ + @Override public Value[] getValueList() { + throw new UnsupportedOperationException(); + } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java index 6671ef9..3817308 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java @@ -24,14 +24,11 @@ import org.h2.value.Value; * Row factory. */ public class GridH2RowFactory extends RowFactory { - /** */ - public static final GridH2RowFactory INSTANCE = new GridH2RowFactory(); - /** * @param v Value. * @return Row. */ - public GridH2Row create(Value v) { + public static GridH2Row create(Value v) { return new RowKey(v); } @@ -40,13 +37,19 @@ public class GridH2RowFactory extends RowFactory { * @param v2 Value 2. * @return Row. */ - public GridH2Row create(Value v1, Value v2) { + public static GridH2Row create(Value v1, Value v2) { return new RowPair(v1, v2); } - /** {@inheritDoc} */ - @Override public GridH2Row createRow(Value[] data, int memory) { + /** + * @param data Values. + * @return Row. + */ + public static GridH2Row create(Value... data) { switch (data.length) { + case 0: + throw new IllegalStateException(); + case 1: return new RowKey(data[0]); @@ -58,6 +61,11 @@ public class GridH2RowFactory extends RowFactory { } } + /** {@inheritDoc} */ + @Override public GridH2Row createRow(Value[] data, int memory) { + return create(data); + } + /** * Single value row. */ @@ -84,13 +92,9 @@ public class GridH2RowFactory extends RowFactory { } /** {@inheritDoc} */ - @Override public void setValue(int index, Value v) { - throw new UnsupportedOperationException(); - } - - /** {@inheritDoc} */ - @Override public Value[] getValueList() { - throw new UnsupportedOperationException(); + @Override public void setValue(int idx, Value v) { + assert idx == 0 : idx; + key = v; } } @@ -124,13 +128,14 @@ public class GridH2RowFactory extends RowFactory { } /** {@inheritDoc} */ - @Override public void setValue(int index, Value v) { - throw new UnsupportedOperationException(); - } + @Override public void setValue(int idx, Value v) { + if (idx == 0) + v1 = v; + else { + assert idx == 1 : idx; - /** {@inheritDoc} */ - @Override public Value[] getValueList() { - throw new UnsupportedOperationException(); + v2 = v; + } } } @@ -149,11 +154,6 @@ public class GridH2RowFactory extends RowFactory { } /** {@inheritDoc} */ - @Override public Value[] getValueList() { - return vals; - } - - /** {@inheritDoc} */ @Override public int getColumnCount() { return vals.length; } @@ -165,7 +165,7 @@ public class GridH2RowFactory extends RowFactory { /** {@inheritDoc} */ @Override public void setValue(int idx, Value v) { - throw new UnsupportedOperationException(); + vals[idx] = v; } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java index 902b67b..db73854 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java @@ -409,7 +409,7 @@ public class GridH2Table extends TableBase { // index(1) is PK, get full row from there (search row here contains only key but no other columns). GridH2Row old = pk.remove(row); - if (old instanceof GridH2AbstractKeyValueRow) { // Unswap value. + if (row.getColumnCount() != 1 && old instanceof GridH2AbstractKeyValueRow) { // Unswap value. Value v = row.getValue(VAL_COL); if (v != null) http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java index 957e5f6..510cb9a 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java @@ -51,7 +51,7 @@ import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TermRangeFilter; import org.apache.lucene.search.TopDocs; import org.apache.lucene.util.Version; -import org.h2.util.Utils; +import org.h2.util.JdbcUtils; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.KEY_FIELD_NAME; @@ -363,7 +363,7 @@ public class GridLuceneIndex implements Closeable { @SuppressWarnings("unchecked") private Z unmarshall(byte[] bytes, ClassLoader ldr) throws IgniteCheckedException { if (coctx == null) // For tests. - return (Z)Utils.deserialize(bytes, null); + return (Z)JdbcUtils.deserialize(bytes, null); return (Z)coctx.processor().unmarshal(coctx, bytes, ldr); } http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/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 index fb2efa1..ee7637d 100644 --- 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 @@ -96,7 +96,7 @@ public class GridMergeIndexUnsorted extends GridMergeIndex { } @Override public Row next() { - return GridH2RowFactory.INSTANCE.createRow(iter.next(), 0); + return GridH2RowFactory.create(iter.next()); } @Override public void remove() { http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java index 194fb82..9b2e2dc 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java @@ -25,7 +25,6 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.concurrent.Callable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.configuration.CacheConfiguration; @@ -44,7 +43,7 @@ import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.testframework.GridStringLogger; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.h2.util.Utils; +import org.h2.util.JdbcUtils; import org.jetbrains.annotations.Nullable; /** @@ -561,7 +560,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract /** {@inheritDoc} */ @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException { - return Utils.serialize(val, null); + return JdbcUtils.serialize(val, null); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/06a17924/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java index e70ee8d..524ee8d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java @@ -120,7 +120,8 @@ public class GridH2TableSelfTest extends GridCommonAbstractTest { * @return New row. */ private GridH2Row row(UUID id, long t, String str, long x) { - return new GridH2Row(ValueUuid.get(id.getMostSignificantBits(), id.getLeastSignificantBits()), + return GridH2RowFactory.create( + ValueUuid.get(id.getMostSignificantBits(), id.getLeastSignificantBits()), ValueTimestamp.get(new Timestamp(t)), ValueString.get(str), ValueLong.get(x));