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 1A9331808A for ; Tue, 3 Nov 2015 09:46:04 +0000 (UTC) Received: (qmail 96989 invoked by uid 500); 3 Nov 2015 09:46:04 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 96956 invoked by uid 500); 3 Nov 2015 09:46:04 -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 96947 invoked by uid 99); 3 Nov 2015 09:46:03 -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; Tue, 03 Nov 2015 09:46:03 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 891E4E03CE; Tue, 3 Nov 2015 09:46:03 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: agoncharuk@apache.org To: commits@ignite.apache.org Date: Tue, 03 Nov 2015 09:46:02 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [2/3] ignite git commit: IGNITE-950 - Fixed tests, renaming. http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index 6251ae5..685b998 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -270,9 +270,30 @@ public final class GridNearGetFuture extends CacheDistributedGetFutureAdap Map savedEntries = null; - // Assign keys to primary nodes. - for (KeyCacheObject key : keys) - savedEntries = map(key, mappings, topVer, mapped, savedEntries); + { + boolean success = false; + + try { + // Assign keys to primary nodes. + for (KeyCacheObject key : keys) + savedEntries = map(key, mappings, topVer, mapped, savedEntries); + + success = true; + } + finally { + // Exception has been thrown, must release reserved near entries. + if (!success) { + GridCacheVersion obsolete = cctx.versions().next(topVer); + + for (GridNearCacheEntry reserved : savedEntries.values()) { + reserved.releaseEviction(); + + if (reserved.markObsolete(obsolete)) + reserved.context().cache().removeEntry(reserved); + } + } + } + } if (isDone()) return; http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java index dcaf14c..08bec85 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java @@ -20,15 +20,15 @@ package org.apache.ignite.internal.processors.cache.portable; import java.util.Collection; import java.util.Map; import org.apache.ignite.IgniteException; -import org.apache.ignite.IgniteObjects; +import org.apache.ignite.IgniteBinary; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; import org.apache.ignite.binary.BinaryObjectBuilder; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryObject; import org.jetbrains.annotations.Nullable; /** - * Extended cache object processor interface with additional methods for portables. + * Extended cache object processor interface with additional methods for binary. */ public interface CacheObjectPortableProcessor extends IgniteCacheObjectProcessor { /** @@ -56,7 +56,7 @@ public interface CacheObjectPortableProcessor extends IgniteCacheObjectProcessor * @param newMeta New meta data. * @throws IgniteException In case of error. */ - public void addMeta(int typeId, final BinaryTypeMetadata newMeta) throws IgniteException; + public void addMeta(int typeId, final BinaryType newMeta) throws IgniteException; /** * @param typeId Type ID. @@ -73,26 +73,26 @@ public interface CacheObjectPortableProcessor extends IgniteCacheObjectProcessor * @return Meta data. * @throws IgniteException In case of error. */ - @Nullable public BinaryTypeMetadata metadata(int typeId) throws IgniteException; + @Nullable public BinaryType metadata(int typeId) throws IgniteException; /** * @param typeIds Type ID. * @return Meta data. * @throws IgniteException In case of error. */ - public Map metadata(Collection typeIds) throws IgniteException; + public Map metadata(Collection typeIds) throws IgniteException; /** * @return Metadata for all types. * @throws IgniteException In case of error. */ - public Collection metadata() throws IgniteException; + public Collection metadata() throws IgniteException; /** * @return Portables interface. * @throws IgniteException If failed. */ - public IgniteObjects portables() throws IgniteException; + public IgniteBinary portables() throws IgniteException; /** * @param obj Original object. http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java index 8e24847..229f7c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java @@ -39,7 +39,7 @@ import javax.cache.processor.EntryProcessor; import javax.cache.processor.MutableEntry; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.IgniteObjects; +import org.apache.ignite.IgniteBinary; import org.apache.ignite.cache.CacheEntryEventSerializableFilter; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; @@ -83,7 +83,7 @@ import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryObject; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -141,10 +141,10 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor private final boolean clientNode; /** */ - private volatile IgniteCacheProxy metaDataCache; + private volatile IgniteCacheProxy metaDataCache; /** */ - private final ConcurrentHashMap8 clientMetaDataCache; + private final ConcurrentHashMap8 clientMetaDataCache; /** Predicate to filter portable meta data in utility cache. */ private final CacheEntryPredicate metaPred = new CacheEntryPredicateAdapter() { @@ -166,10 +166,10 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor /** */ @GridToStringExclude - private IgniteObjects portables; + private IgniteBinary portables; /** Metadata updates collected before metadata cache is initialized. */ - private final Map metaBuf = new ConcurrentHashMap<>(); + private final Map metaBuf = new ConcurrentHashMap<>(); /** */ private UUID metaCacheQryId; @@ -268,17 +268,17 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor clientNode = this.ctx.clientNode(); - clientMetaDataCache = clientNode ? new ConcurrentHashMap8() : null; + clientMetaDataCache = clientNode ? new ConcurrentHashMap8() : null; } /** {@inheritDoc} */ @Override public void start() throws IgniteCheckedException { if (marsh instanceof PortableMarshaller) { PortableMetaDataHandler metaHnd = new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryTypeMetadata newMeta) + @Override public void addMeta(int typeId, BinaryType newMeta) throws BinaryObjectException { if (metaDataCache == null) { - BinaryTypeMetadata oldMeta = metaBuf.get(typeId); + BinaryType oldMeta = metaBuf.get(typeId); if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta, null)) { synchronized (this) { @@ -307,7 +307,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor CacheObjectPortableProcessorImpl.this.addMeta(typeId, newMeta); } - @Override public BinaryTypeMetadata metadata(int typeId) throws BinaryObjectException { + @Override public BinaryType metadata(int typeId) throws BinaryObjectException { if (metaDataCache == null) U.awaitQuiet(startLatch); @@ -323,7 +323,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor portableMarsh = new GridPortableMarshaller(portableCtx); - portables = new IgniteObjectsImpl(ctx, this); + portables = new IgniteBinaryImpl(ctx, this); } } @@ -350,7 +350,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor GridCacheQueryManager qryMgr = metaDataCache.context().queries(); - CacheQuery> qry = + CacheQuery> qry = qryMgr.createScanQuery(new MetaDataPredicate(), null, false); qry.keepAll(false); @@ -358,9 +358,9 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor qry.projection(ctx.cluster().get().forNode(oldestSrvNode)); try { - CacheQueryFuture> fut = qry.execute(); + CacheQueryFuture> fut = qry.execute(); - Map.Entry next; + Map.Entry next; while ((next = fut.next()) != null) { assert next.getKey() != null : next; @@ -382,7 +382,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor startLatch.countDown(); - for (Map.Entry e : metaBuf.entrySet()) + for (Map.Entry e : metaBuf.entrySet()) addMeta(e.getKey(), e.getValue()); metaBuf.clear(); @@ -400,11 +400,11 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor * @param key Metadata key. * @param newMeta Metadata. */ - private void addClientCacheMetaData(PortableMetaDataKey key, final BinaryTypeMetadata newMeta) { + private void addClientCacheMetaData(PortableMetaDataKey key, final BinaryType newMeta) { clientMetaDataCache.compute(key, - new ConcurrentHashMap8.BiFun() { - @Override public BinaryTypeMetadata apply(PortableMetaDataKey key, BinaryTypeMetadata oldMeta) { - BinaryTypeMetadata res; + new ConcurrentHashMap8.BiFun() { + @Override public BinaryType apply(PortableMetaDataKey key, BinaryType oldMeta) { + BinaryType res; try { res = checkMeta(key.typeId(), oldMeta, newMeta, null) ? newMeta : oldMeta; @@ -561,13 +561,13 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor } /** {@inheritDoc} */ - @Override public void addMeta(final int typeId, final BinaryTypeMetadata newMeta) throws BinaryObjectException { + @Override public void addMeta(final int typeId, final BinaryType newMeta) throws BinaryObjectException { assert newMeta != null; final PortableMetaDataKey key = new PortableMetaDataKey(typeId); try { - BinaryTypeMetadata oldMeta = metaDataCache.localPeek(key); + BinaryType oldMeta = metaDataCache.localPeek(key); if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta, null)) { BinaryObjectException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta)); @@ -582,7 +582,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor } /** {@inheritDoc} */ - @Nullable @Override public BinaryTypeMetadata metadata(final int typeId) throws BinaryObjectException { + @Nullable @Override public BinaryType metadata(final int typeId) throws BinaryObjectException { try { if (clientNode) return clientMetaDataCache.get(new PortableMetaDataKey(typeId)); @@ -595,7 +595,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor } /** {@inheritDoc} */ - @Override public Map metadata(Collection typeIds) + @Override public Map metadata(Collection typeIds) throws BinaryObjectException { try { Collection keys = new ArrayList<>(typeIds.size()); @@ -603,11 +603,11 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor for (Integer typeId : typeIds) keys.add(new PortableMetaDataKey(typeId)); - Map meta = metaDataCache.getAll(keys); + Map meta = metaDataCache.getAll(keys); - Map res = U.newHashMap(meta.size()); + Map res = U.newHashMap(meta.size()); - for (Map.Entry e : meta.entrySet()) + for (Map.Entry e : meta.entrySet()) res.put(e.getKey().typeId(), e.getValue()); return res; @@ -619,23 +619,23 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public Collection metadata() throws BinaryObjectException { + @Override public Collection metadata() throws BinaryObjectException { if (clientNode) return new ArrayList<>(clientMetaDataCache.values()); return F.viewReadOnly(metaDataCache.entrySetx(metaPred), - new C1, BinaryTypeMetadata>() { + new C1, BinaryType>() { private static final long serialVersionUID = 0L; - @Override public BinaryTypeMetadata apply( - Cache.Entry e) { + @Override public BinaryType apply( + Cache.Entry e) { return e.getValue(); } }); } /** {@inheritDoc} */ - @Override public IgniteObjects portables() throws IgniteException { + @Override public IgniteBinary portables() throws IgniteException { return portables; } @@ -655,7 +655,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor */ public Object affinityKey(BinaryObject po) { try { - BinaryTypeMetadata meta = po.metaData(); + BinaryType meta = po.metaData(); if (meta != null) { String affKeyFieldName = meta.affinityKeyFieldName(); @@ -831,8 +831,8 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor * @return Whether meta is changed. * @throws org.apache.ignite.binary.BinaryObjectException In case of error. */ - private static boolean checkMeta(int typeId, @Nullable BinaryTypeMetadata oldMeta, - BinaryTypeMetadata newMeta, @Nullable Map fields) throws BinaryObjectException { + private static boolean checkMeta(int typeId, @Nullable BinaryType oldMeta, + BinaryType newMeta, @Nullable Map fields) throws BinaryObjectException { assert newMeta != null; Map oldFields = oldMeta != null ? ((BinaryMetaDataImpl)oldMeta).fieldsMeta() : null; @@ -896,7 +896,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor /** */ private static class MetaDataProcessor implements - EntryProcessor, Externalizable { + EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -904,7 +904,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor private int typeId; /** */ - private BinaryTypeMetadata newMeta; + private BinaryType newMeta; /** * For {@link Externalizable}. @@ -917,7 +917,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor * @param typeId Type ID. * @param newMeta New metadata. */ - private MetaDataProcessor(int typeId, BinaryTypeMetadata newMeta) { + private MetaDataProcessor(int typeId, BinaryType newMeta) { assert newMeta != null; this.typeId = typeId; @@ -926,15 +926,15 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor /** {@inheritDoc} */ @Override public BinaryObjectException process( - MutableEntry entry, + MutableEntry entry, Object... args) { try { - BinaryTypeMetadata oldMeta = entry.getValue(); + BinaryType oldMeta = entry.getValue(); Map fields = new HashMap<>(); if (checkMeta(typeId, oldMeta, newMeta, fields)) { - BinaryTypeMetadata res = new BinaryMetaDataImpl(newMeta.typeName(), + BinaryType res = new BinaryMetaDataImpl(newMeta.typeName(), fields, newMeta.affinityKeyFieldName()); @@ -959,7 +959,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor /** {@inheritDoc} */ @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { typeId = in.readInt(); - newMeta = (BinaryTypeMetadata)in.readObject(); + newMeta = (BinaryType)in.readObject(); } /** {@inheritDoc} */ @@ -971,17 +971,17 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor /** * */ - class MetaDataEntryListener implements CacheEntryUpdatedListener { + class MetaDataEntryListener implements CacheEntryUpdatedListener { /** {@inheritDoc} */ @Override public void onUpdated( - Iterable> evts) + Iterable> evts) throws CacheEntryListenerException { - for (CacheEntryEvent evt : evts) { + for (CacheEntryEvent evt : evts) { assert evt.getEventType() == EventType.CREATED || evt.getEventType() == EventType.UPDATED : evt; PortableMetaDataKey key = evt.getKey(); - final BinaryTypeMetadata newMeta = evt.getValue(); + final BinaryType newMeta = evt.getValue(); assert newMeta != null : evt; http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteBinaryImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteBinaryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteBinaryImpl.java new file mode 100644 index 0000000..5ef24d8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteBinaryImpl.java @@ -0,0 +1,177 @@ +/* + * 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.cache.portable; + +import java.util.Collection; +import org.apache.ignite.IgniteBinary; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; +import org.apache.ignite.binary.BinaryObjectBuilder; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.binary.BinaryObject; +import org.jetbrains.annotations.Nullable; + +/** + * {@link org.apache.ignite.IgniteBinary} implementation. + */ +public class IgniteBinaryImpl implements IgniteBinary { + /** */ + private GridKernalContext ctx; + + /** */ + private CacheObjectPortableProcessor proc; + + /** + * @param ctx Context. + */ + public IgniteBinaryImpl(GridKernalContext ctx, CacheObjectPortableProcessor proc) { + this.ctx = ctx; + + this.proc = proc; + } + + /** {@inheritDoc} */ + @Override public int typeId(String typeName) { + guard(); + + try { + return proc.typeId(typeName); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public T toBinary(@Nullable Object obj) throws BinaryObjectException { + guard(); + + try { + return (T)proc.marshalToPortable(obj); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public BinaryObjectBuilder builder(int typeId) { + guard(); + + try { + return proc.builder(typeId); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public BinaryObjectBuilder builder(String typeName) { + guard(); + + try { + return proc.builder(typeName); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public BinaryObjectBuilder builder(BinaryObject portableObj) { + guard(); + + try { + return proc.builder(portableObj); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Nullable @Override public BinaryType metadata(Class cls) throws BinaryObjectException { + guard(); + + try { + return proc.metadata(proc.typeId(cls.getName())); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Nullable @Override public BinaryType metadata(String typeName) throws BinaryObjectException { + guard(); + + try { + return proc.metadata(proc.typeId(typeName)); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Nullable @Override public BinaryType metadata(int typeId) throws BinaryObjectException { + guard(); + + try { + return proc.metadata(typeId); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public Collection metadata() throws BinaryObjectException { + guard(); + + try { + return proc.metadata(); + } + finally { + unguard(); + } + } + + /** + * @return Portable processor. + */ + public IgniteCacheObjectProcessor processor() { + return proc; + } + + /** + * ctx.gateway().readLock() + */ + private void guard() { + ctx.gateway().readLock(); + } + + /** + * ctx.gateway().readUnlock() + */ + private void unguard() { + ctx.gateway().readUnlock(); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteObjectsImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteObjectsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteObjectsImpl.java deleted file mode 100644 index 225eabb..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteObjectsImpl.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * 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.cache.portable; - -import java.util.Collection; -import org.apache.ignite.IgniteObjects; -import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; -import org.apache.ignite.binary.BinaryObjectBuilder; -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryTypeMetadata; -import org.apache.ignite.binary.BinaryObject; -import org.jetbrains.annotations.Nullable; - -/** - * {@link org.apache.ignite.IgniteObjects} implementation. - */ -public class IgniteObjectsImpl implements IgniteObjects { - /** */ - private GridKernalContext ctx; - - /** */ - private CacheObjectPortableProcessor proc; - - /** - * @param ctx Context. - */ - public IgniteObjectsImpl(GridKernalContext ctx, CacheObjectPortableProcessor proc) { - this.ctx = ctx; - - this.proc = proc; - } - - /** {@inheritDoc} */ - @Override public int typeId(String typeName) { - guard(); - - try { - return proc.typeId(typeName); - } - finally { - unguard(); - } - } - - /** {@inheritDoc} */ - @Override public T toPortable(@Nullable Object obj) throws BinaryObjectException { - guard(); - - try { - return (T)proc.marshalToPortable(obj); - } - finally { - unguard(); - } - } - - /** {@inheritDoc} */ - @Override public BinaryObjectBuilder builder(int typeId) { - guard(); - - try { - return proc.builder(typeId); - } - finally { - unguard(); - } - } - - /** {@inheritDoc} */ - @Override public BinaryObjectBuilder builder(String typeName) { - guard(); - - try { - return proc.builder(typeName); - } - finally { - unguard(); - } - } - - /** {@inheritDoc} */ - @Override public BinaryObjectBuilder builder(BinaryObject portableObj) { - guard(); - - try { - return proc.builder(portableObj); - } - finally { - unguard(); - } - } - - /** {@inheritDoc} */ - @Nullable @Override public BinaryTypeMetadata metadata(Class cls) throws BinaryObjectException { - guard(); - - try { - return proc.metadata(proc.typeId(cls.getName())); - } - finally { - unguard(); - } - } - - /** {@inheritDoc} */ - @Nullable @Override public BinaryTypeMetadata metadata(String typeName) throws BinaryObjectException { - guard(); - - try { - return proc.metadata(proc.typeId(typeName)); - } - finally { - unguard(); - } - } - - /** {@inheritDoc} */ - @Nullable @Override public BinaryTypeMetadata metadata(int typeId) throws BinaryObjectException { - guard(); - - try { - return proc.metadata(typeId); - } - finally { - unguard(); - } - } - - /** {@inheritDoc} */ - @Override public Collection metadata() throws BinaryObjectException { - guard(); - - try { - return proc.metadata(); - } - finally { - unguard(); - } - } - - /** - * @return Portable processor. - */ - public IgniteCacheObjectProcessor processor() { - return proc; - } - - /** - * ctx.gateway().readLock() - */ - private void guard() { - ctx.gateway().readLock(); - } - - /** - * ctx.gateway().readUnlock() - */ - private void unguard() { - ctx.gateway().readUnlock(); - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java index c7feda4..fa2d6b1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java @@ -165,7 +165,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache * @param incBackups {@code true} if need to include backups. * @param fields Fields query flag. * @param all Whether to load all pages. - * @param keepPortable Whether to keep portables. + * @param keepPortable Whether to keep binary. * @param subjId Subject ID. * @param taskHash Task name hash code. * @param topVer Topology version. @@ -452,7 +452,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache } /** - * @return Whether to keep portables. + * @return Whether to keep binary. */ public boolean keepPortable() { return keepPortable; http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java index b12ccf9..f48ff38 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java @@ -758,7 +758,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt assert e != null; if (e.getMessage() != null) { - throw new IgniteCheckedException("Cache store must work with portable objects if portables are " + + throw new IgniteCheckedException("Cache store must work with portable objects if binary are " + "enabled for cache [cacheName=" + cctx.namex() + ']', e); } else http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index a0648ca..214ba45 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -3185,7 +3185,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter * Checks if portable values should be deserialized. * * @param cacheCtx Cache context. - * @return {@code True} if portables should be deserialized, {@code false} otherwise. + * @return {@code True} if binary should be deserialized, {@code false} otherwise. */ private boolean deserializePortables(GridCacheContext cacheCtx) { CacheOperationContext opCtx = cacheCtx.operationContextPerCall(); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java index ed6be20..4a1b41a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java @@ -70,7 +70,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T4; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.jetbrains.annotations.Nullable; import java.sql.Timestamp; @@ -370,11 +370,11 @@ public class PlatformContextImpl implements PlatformContext { /** {@inheritDoc} */ @Override public void writeAllMetadata(BinaryRawWriterEx writer) { - Collection metas = cacheObjProc.metadata(); + Collection metas = cacheObjProc.metadata(); writer.writeInt(metas.size()); - for (BinaryTypeMetadata m : metas) + for (BinaryType m : metas) writeMetadata0(writer, cacheObjProc.typeId(m.typeName()), m); } @@ -385,7 +385,7 @@ public class PlatformContextImpl implements PlatformContext { * @param typeId Type id. * @param meta Metadata. */ - private void writeMetadata0(BinaryRawWriterEx writer, int typeId, BinaryTypeMetadata meta) { + private void writeMetadata0(BinaryRawWriterEx writer, int typeId, BinaryType meta) { if (meta == null) writer.writeBoolean(false); else { http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java index 11d4c74..05db49d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java @@ -295,7 +295,7 @@ public class PlatformCompute extends PlatformAbstractTarget { * @return Result. */ private Object toPortable(Object src) { - return platformCtx.kernalContext().grid().portables().toPortable(src); + return platformCtx.kernalContext().grid().binary().toBinary(src); } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java index c49c337..60c0693 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java @@ -40,7 +40,7 @@ import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.platform.dotnet.PlatformDotNetLifecycleBean; import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import java.util.ArrayList; import java.util.Collections; @@ -229,12 +229,12 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur private static GridPortableMarshaller marshaller() { try { PortableContext ctx = new PortableContext(new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryTypeMetadata meta) + @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException { // No-op. } - @Override public BinaryTypeMetadata metadata(int typeId) throws BinaryObjectException { + @Override public BinaryType metadata(int typeId) throws BinaryObjectException { return null; } }, new IgniteConfiguration()); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientTaskRequest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientTaskRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientTaskRequest.java index 6b53143..3ac16f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientTaskRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientTaskRequest.java @@ -35,7 +35,7 @@ public class GridClientTaskRequest extends GridClientAbstractMessage { /** Task parameter. */ private Object arg; - /** Keep portables flag. */ + /** Keep binary flag. */ private boolean keepPortables; /** @@ -67,14 +67,14 @@ public class GridClientTaskRequest extends GridClientAbstractMessage { } /** - * @return Keep portables flag. + * @return Keep binary flag. */ public boolean keepPortables() { return keepPortables; } /** - * @param keepPortables Keep portables flag. + * @param keepPortables Keep binary flag. */ public void keepPortables(boolean keepPortables) { this.keepPortables = keepPortables; http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java index 54e61f3..e3adfc1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderAdditionalSelfTest.java @@ -36,18 +36,18 @@ import java.util.Objects; import java.util.Set; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteObjects; +import org.apache.ignite.IgniteBinary; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.portable.builder.PortableBuilderEnum; import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.portable.mutabletest.GridBinaryMarshalerAwareTestClass; import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl; -import org.apache.ignite.internal.processors.cache.portable.IgniteObjectsImpl; +import org.apache.ignite.internal.processors.cache.portable.IgniteBinaryImpl; import org.apache.ignite.internal.util.lang.GridMapEntry; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryObjectBuilder; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -114,8 +114,8 @@ public class GridBinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstrac /** * @return Portables API. */ - protected IgniteObjects portables() { - return grid(0).portables(); + protected IgniteBinary portables() { + return grid(0).binary(); } /** @@ -965,7 +965,7 @@ public class GridBinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstrac mutableObj.build(); - BinaryTypeMetadata metadata = portables().metadata(TestObjectContainer.class); + BinaryType metadata = portables().metadata(TestObjectContainer.class); assertEquals("String", metadata.fieldTypeName("xx567")); } @@ -981,7 +981,7 @@ public class GridBinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstrac mutableObj.build(); - BinaryTypeMetadata metadata = portables().metadata(TestObjectContainer.class); + BinaryType metadata = portables().metadata(TestObjectContainer.class); assertEquals("String", metadata.fieldTypeName("xx567")); } @@ -1005,7 +1005,7 @@ public class GridBinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstrac mutableObj.build(); - BinaryTypeMetadata metadata = portables().metadata(c.getClass()); + BinaryType metadata = portables().metadata(c.getClass()); assertTrue(metadata.fields().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField", "colField", "mapField", "enumField", "enumArrField"))); @@ -1264,7 +1264,7 @@ public class GridBinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstrac * @return Object in portable format. */ private BinaryObject toPortable(Object obj) { - return portables().toPortable(obj); + return portables().toBinary(obj); } /** @@ -1281,7 +1281,7 @@ public class GridBinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstrac */ private BinaryObjectBuilderImpl newWrapper(Class aCls) { CacheObjectPortableProcessorImpl processor = (CacheObjectPortableProcessorImpl)( - (IgniteObjectsImpl)portables()).processor(); + (IgniteBinaryImpl)portables()).processor(); return new BinaryObjectBuilderImpl(processor.portableContext(), processor.typeId(aCls.getName()), aCls.getSimpleName()); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java index 8f025e4..1c19639 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridBinaryObjectBuilderSelfTest.java @@ -28,7 +28,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteObjects; +import org.apache.ignite.IgniteBinary; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes; @@ -42,7 +42,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryTypeIdMapper; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.testframework.GridTestUtils; @@ -775,7 +775,7 @@ public class GridBinaryObjectBuilderSelfTest extends GridCommonAbstractTest { BinaryObject po = builder.build(); - BinaryTypeMetadata meta = po.metaData(); + BinaryType meta = po.metaData(); assertEquals("MetaTest2", meta.typeName()); assertEquals("Object", meta.fieldTypeName("objectField")); @@ -794,7 +794,7 @@ public class GridBinaryObjectBuilderSelfTest extends GridCommonAbstractTest { BinaryObject po = builder.build(); - BinaryTypeMetadata meta = po.metaData(); + BinaryType meta = po.metaData(); assertEquals("MetaTest", meta.typeName()); @@ -965,8 +965,8 @@ public class GridBinaryObjectBuilderSelfTest extends GridCommonAbstractTest { /** * @return Portables. */ - private IgniteObjects portables() { - return grid(0).portables(); + private IgniteBinary portables() { + return grid(0).binary(); } /** @@ -974,7 +974,7 @@ public class GridBinaryObjectBuilderSelfTest extends GridCommonAbstractTest { * @return Portable object. */ private BinaryObject toPortable(Object obj) { - return portables().toPortable(obj); + return portables().toBinary(obj); } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java index 3fd62c1..ee91167 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java @@ -29,7 +29,7 @@ import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.Binarylizable; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryReader; import org.apache.ignite.binary.BinaryWriter; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -40,11 +40,11 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstractTest { /** */ protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryTypeMetadata meta) { + @Override public void addMeta(int typeId, BinaryType meta) { // No-op. } - @Override public BinaryTypeMetadata metadata(int typeId) { + @Override public BinaryType metadata(int typeId) { return null; } }; http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java index 001032e..7576103 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java @@ -47,7 +47,7 @@ import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.BinaryTypeIdMapper; import org.apache.ignite.binary.Binarylizable; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.binary.BinaryRawWriter; import org.apache.ignite.binary.BinaryReader; @@ -83,11 +83,11 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest { /** */ protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryTypeMetadata meta) { + @Override public void addMeta(int typeId, BinaryType meta) { // No-op. } - @Override public BinaryTypeMetadata metadata(int typeId) { + @Override public BinaryType metadata(int typeId) { return null; } }; http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java index ed7190e..fbdb562 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java @@ -17,7 +17,7 @@ package org.apache.ignite.internal.portable; import java.util.Arrays; -import org.apache.ignite.IgniteObjects; +import org.apache.ignite.IgniteBinary; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryObjectBuilder; @@ -47,8 +47,8 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest /** * @return Portables. */ - private IgniteObjects portables() { - return grid().portables(); + private IgniteBinary portables() { + return grid().binary(); } /** @@ -67,9 +67,9 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest try { startGrid(); - portables().toPortable(new TestObject1()); - portables().toPortable(new TestObject2()); - portables().toPortable(new TestObject3()); + portables().toBinary(new TestObject1()); + portables().toBinary(new TestObject2()); + portables().toBinary(new TestObject3()); assertEquals(0, portables().metadata(TestObject1.class).fields().size()); assertEquals(0, portables().metadata(TestObject2.class).fields().size()); @@ -106,8 +106,8 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest try { startGrid(); - portables().toPortable(new TestObject1()); - portables().toPortable(new TestObject2()); + portables().toBinary(new TestObject1()); + portables().toBinary(new TestObject2()); assertEquals(0, portables().metadata(TestObject1.class).fields().size()); assertEquals(1, portables().metadata(TestObject2.class).fields().size()); @@ -137,8 +137,8 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest try { startGrid(); - portables().toPortable(new TestObject1()); - portables().toPortable(new TestObject2()); + portables().toBinary(new TestObject1()); + portables().toBinary(new TestObject2()); assertEquals(1, portables().metadata(TestObject1.class).fields().size()); assertEquals(0, portables().metadata(TestObject2.class).fields().size()); @@ -166,8 +166,8 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest try { startGrid(); - portables().toPortable(new TestObject1()); - portables().toPortable(new TestObject2()); + portables().toBinary(new TestObject1()); + portables().toBinary(new TestObject2()); assertEquals(0, portables().metadata(TestObject1.class).fields().size()); assertEquals(1, portables().metadata(TestObject2.class).fields().size()); @@ -195,8 +195,8 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest try { startGrid(); - portables().toPortable(new TestObject1()); - portables().toPortable(new TestObject2()); + portables().toBinary(new TestObject1()); + portables().toBinary(new TestObject2()); assertEquals(1, portables().metadata(TestObject1.class).fields().size()); assertEquals(0, portables().metadata(TestObject2.class).fields().size()); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java index 5b54fb0..98cb9cd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java @@ -22,13 +22,13 @@ import java.util.Arrays; import java.util.Collection; import java.util.Date; import java.util.HashMap; -import org.apache.ignite.IgniteObjects; +import org.apache.ignite.IgniteBinary; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.Binarylizable; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryRawWriter; import org.apache.ignite.binary.BinaryReader; @@ -74,21 +74,21 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { /** * @return Portables API. */ - protected IgniteObjects portables() { - return grid().portables(); + protected IgniteBinary portables() { + return grid().binary(); } /** * @throws Exception If failed. */ public void testGetAll() throws Exception { - portables().toPortable(new TestObject2()); + portables().toBinary(new TestObject2()); - Collection metas = portables().metadata(); + Collection metas = portables().metadata(); assertEquals(2, metas.size()); - for (BinaryTypeMetadata meta : metas) { + for (BinaryType meta : metas) { Collection fields; switch (meta.typeName()) { @@ -150,7 +150,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { public void testNoConfiguration() throws Exception { fail("https://issues.apache.org/jira/browse/IGNITE-1377"); - portables().toPortable(new TestObject3()); + portables().toBinary(new TestObject3()); assertNotNull(portables().metadata(TestObject3.class)); } @@ -159,7 +159,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testReflection() throws Exception { - BinaryTypeMetadata meta = portables().metadata(TestObject1.class); + BinaryType meta = portables().metadata(TestObject1.class); assertNotNull(meta); @@ -190,9 +190,9 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testPortableMarshalAware() throws Exception { - portables().toPortable(new TestObject2()); + portables().toBinary(new TestObject2()); - BinaryTypeMetadata meta = portables().metadata(TestObject2.class); + BinaryType meta = portables().metadata(TestObject2.class); assertNotNull(meta); @@ -223,13 +223,13 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testMerge() throws Exception { - portables().toPortable(new TestObject2()); + portables().toBinary(new TestObject2()); idx = 1; - portables().toPortable(new TestObject2()); + portables().toBinary(new TestObject2()); - BinaryTypeMetadata meta = portables().metadata(TestObject2.class); + BinaryType meta = portables().metadata(TestObject2.class); assertNotNull(meta); @@ -274,11 +274,11 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { obj.decVal = BigDecimal.ZERO; obj.decArrVal = new BigDecimal[] { BigDecimal.ONE }; - BinaryObject po = portables().toPortable(obj); + BinaryObject po = portables().toBinary(obj); info(po.toString()); - BinaryTypeMetadata meta = po.metaData(); + BinaryType meta = po.metaData(); assertNotNull(meta); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java index fe34fdc..22125af 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java @@ -24,7 +24,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.MarshallerContextTestImpl; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryTypeIdMapper; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -34,11 +34,11 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest { /** */ private static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryTypeMetadata meta) { + @Override public void addMeta(int typeId, BinaryType meta) { // No-op. } - @Override public BinaryTypeMetadata metadata(int typeId) { + @Override public BinaryType metadata(int typeId) { return null; } }; http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java index c25840b..0892750 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java @@ -29,7 +29,7 @@ import javax.cache.processor.EntryProcessor; import javax.cache.processor.MutableEntry; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteObjects; +import org.apache.ignite.IgniteBinary; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CachePeekMode; @@ -836,7 +836,7 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA Ignite ignite = e.unwrap(Ignite.class); - IgniteObjects portables = ignite.portables(); + IgniteBinary portables = ignite.binary(); BinaryObjectBuilder builder = portables.builder(val); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java index eddf066..a33eb7b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java @@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteObjects; +import org.apache.ignite.IgniteBinary; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; @@ -35,7 +35,7 @@ import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryObjectBuilder; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -99,7 +99,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo // Update portable metadata concurrently with client nodes start. fut = GridTestUtils.runMultiThreadedAsync(new Callable() { @Override public Object call() throws Exception { - IgniteObjects portables = ignite(0).portables(); + IgniteBinary portables = ignite(0).binary(); IgniteCache cache = ignite(0).cache(null).withKeepBinary(); @@ -156,15 +156,15 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo assertEquals((Object)client, ignite(i).configuration().isClientMode()); - IgniteObjects portables = ignite(i).portables(); + IgniteBinary portables = ignite(i).binary(); - Collection metaCol = portables.metadata(); + Collection metaCol = portables.metadata(); assertEquals(allTypes.size(), metaCol.size()); Set names = new HashSet<>(); - for (BinaryTypeMetadata meta : metaCol) { + for (BinaryType meta : metaCol) { assertTrue(names.add(meta.typeName())); assertNull(meta.affinityKeyFieldName()); @@ -182,7 +182,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo public void testFailoverOnStart() throws Exception { startGrids(4); - IgniteObjects portables = ignite(0).portables(); + IgniteBinary portables = ignite(0).binary(); IgniteCache cache = ignite(0).cache(null).withKeepBinary(); @@ -232,25 +232,25 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo assertEquals((Object) client, ignite(i).configuration().isClientMode()); - portables = ignite(i).portables(); + portables = ignite(i).binary(); - final IgniteObjects p0 = portables; + final IgniteBinary p0 = portables; GridTestUtils.waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { - Collection metaCol = p0.metadata(); + Collection metaCol = p0.metadata(); return metaCol.size() == 1000; } }, getTestTimeout()); - Collection metaCol = portables.metadata(); + Collection metaCol = portables.metadata(); assertEquals(1000, metaCol.size()); Set names = new HashSet<>(); - for (BinaryTypeMetadata meta : metaCol) { + for (BinaryType meta : metaCol) { assertTrue(names.add(meta.typeName())); assertNull(meta.affinityKeyFieldName()); @@ -278,7 +278,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo assertFalse(ignite1.configuration().isClientMode()); - IgniteObjects portables = ignite(1).portables(); + IgniteBinary portables = ignite(1).binary(); IgniteCache cache = ignite(1).cache(null).withKeepBinary(); @@ -290,6 +290,6 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo cache.put(i, builder.build()); } - assertEquals(100, ignite(0).portables().metadata().size()); + assertEquals(100, ignite(0).binary().metadata().size()); } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java index c2464db..b842ebb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java @@ -30,7 +30,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryObjectBuilder; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -117,7 +117,7 @@ public class GridCacheClientNodeBinaryObjectMetadataTest extends GridCacheAbstra } { - BinaryObjectBuilder builder = ignite0.portables().builder("TestObject3"); + BinaryObjectBuilder builder = ignite0.binary().builder("TestObject3"); builder.setField("f1", 1); @@ -127,24 +127,24 @@ public class GridCacheClientNodeBinaryObjectMetadataTest extends GridCacheAbstra BinaryObject obj = cache.get(0); - BinaryTypeMetadata meta = obj.metaData(); + BinaryType meta = obj.metaData(); assertNotNull(meta); assertEquals(1, meta.fields().size()); - meta = ignite0.portables().metadata(TestObject1.class); + meta = ignite0.binary().metadata(TestObject1.class); assertNotNull(meta); assertEquals("val2", meta.affinityKeyFieldName()); - meta = ignite0.portables().metadata(TestObject2.class); + meta = ignite0.binary().metadata(TestObject2.class); assertNotNull(meta); assertNull(meta.affinityKeyFieldName()); } { - BinaryObjectBuilder builder = ignite1.portables().builder("TestObject3"); + BinaryObjectBuilder builder = ignite1.binary().builder("TestObject3"); builder.setField("f2", 2); @@ -154,23 +154,23 @@ public class GridCacheClientNodeBinaryObjectMetadataTest extends GridCacheAbstra BinaryObject obj = cache.get(0); - BinaryTypeMetadata meta = obj.metaData(); + BinaryType meta = obj.metaData(); assertNotNull(meta); assertEquals(2, meta.fields().size()); - meta = ignite1.portables().metadata(TestObject1.class); + meta = ignite1.binary().metadata(TestObject1.class); assertNotNull(meta); assertEquals("val2", meta.affinityKeyFieldName()); - meta = ignite1.portables().metadata(TestObject2.class); + meta = ignite1.binary().metadata(TestObject2.class); assertNotNull(meta); assertNull(meta.affinityKeyFieldName()); } - BinaryTypeMetadata meta = ignite0.portables().metadata("TestObject3"); + BinaryType meta = ignite0.binary().metadata("TestObject3"); assertNotNull(meta); assertEquals(2, meta.fields().size()); @@ -187,15 +187,15 @@ public class GridCacheClientNodeBinaryObjectMetadataTest extends GridCacheAbstra assertNotNull(meta); assertEquals(2, meta.fields().size()); - Collection meta1 = ignite1.portables().metadata(); - Collection meta2 = ignite1.portables().metadata(); + Collection meta1 = ignite1.binary().metadata(); + Collection meta2 = ignite1.binary().metadata(); assertEquals(meta1.size(), meta2.size()); - for (BinaryTypeMetadata m1 : meta1) { + for (BinaryType m1 : meta1) { boolean found = false; - for (BinaryTypeMetadata m2 : meta1) { + for (BinaryType m2 : meta1) { if (m1.typeName().equals(m2.typeName())) { assertEquals(m1.affinityKeyFieldName(), m2.affinityKeyFieldName()); assertEquals(m1.fields(), m2.fields()); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java index 7c605b5..1c842e1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java @@ -36,7 +36,7 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; /** - * Tests for cache store with portables. + * Tests for cache store with binary. */ public abstract class GridCachePortableStoreAbstractSelfTest extends GridCommonAbstractTest { /** */ @@ -76,7 +76,7 @@ public abstract class GridCachePortableStoreAbstractSelfTest extends GridCommonA } /** - * @return Keep portables in store flag. + * @return Keep binary in store flag. */ protected abstract boolean keepPortableInStore(); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreObjectsSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreObjectsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreObjectsSelfTest.java index 1c1c99e..8f81572 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreObjectsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreObjectsSelfTest.java @@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.portable; import java.util.Map; /** - * Tests for cache store with portables. + * Tests for cache store with binary. */ public class GridCachePortableStoreObjectsSelfTest extends GridCachePortableStoreAbstractSelfTest { /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java index 492c4ca..4471502 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java @@ -20,7 +20,7 @@ import java.util.Map; import org.apache.ignite.binary.BinaryObject; /** - * Tests for cache store with portables. + * Tests for cache store with binary. */ public class GridCachePortableStorePortablesSelfTest extends GridCachePortableStoreAbstractSelfTest { /** {@inheritDoc} */ @@ -61,6 +61,6 @@ public class GridCachePortableStorePortablesSelfTest extends GridCachePortableSt * @return Portable object. */ private Object portable(Object obj) { - return grid().portables().toPortable(obj); + return grid().binary().toBinary(obj); } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java index d2a7385..d277801 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java @@ -27,7 +27,7 @@ import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerContextTestImpl; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryTypeMetadata; +import org.apache.ignite.binary.BinaryType; /** * @@ -40,11 +40,11 @@ public class GridPortableCacheEntryMemorySizeSelfTest extends GridCacheEntryMemo marsh.setContext(new MarshallerContextTestImpl(null)); PortableContext pCtx = new PortableContext(new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryTypeMetadata meta) throws BinaryObjectException { + @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException { // No-op } - @Override public BinaryTypeMetadata metadata(int typeId) throws BinaryObjectException { + @Override public BinaryType metadata(int typeId) throws BinaryObjectException { return null; } }, new IgniteConfiguration()); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java index 699ed83..3bc3575 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java @@ -54,7 +54,7 @@ public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest { /** Number of keys to load via data streamer. */ private static final int KEYS_COUNT = 1000; - /** Flag indicating should be cache configured with portables or not. */ + /** Flag indicating should be cache configured with binary or not. */ private static boolean portables; /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java index c845257..0c6b77c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java @@ -33,7 +33,7 @@ public class GridCacheOffHeapTieredAtomicPortableSelfTest extends GridCacheOffHe /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - // Enable portables. + // Enable binary. IgniteConfiguration cfg = super.getConfiguration(gridName); PortableMarshaller marsh = new PortableMarshaller(); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java index 46bbad1..215567c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java @@ -29,7 +29,7 @@ import org.apache.ignite.binary.BinaryObject; public class GridCacheOffHeapTieredEvictionAtomicPortableSelfTest extends GridCacheOffHeapTieredEvictionAtomicSelfTest { /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - // Enable portables. + // Enable binary. IgniteConfiguration cfg = super.getConfiguration(gridName); PortableMarshaller marsh = new PortableMarshaller(); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java index d321da4..674e17a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java @@ -29,7 +29,7 @@ import org.apache.ignite.binary.BinaryObject; public class GridCacheOffHeapTieredEvictionPortableSelfTest extends GridCacheOffHeapTieredEvictionSelfTest { /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - // Enable portables. + // Enable binary. IgniteConfiguration cfg = super.getConfiguration(gridName); PortableMarshaller marsh = new PortableMarshaller(); http://git-wip-us.apache.org/repos/asf/ignite/blob/2a1daa20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java index 6170e39..9f4e2c5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java @@ -33,7 +33,7 @@ public class GridCacheOffHeapTieredPortableSelfTest extends GridCacheOffHeapTier /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - // Enable portables. + // Enable binary. IgniteConfiguration cfg = super.getConfiguration(gridName); PortableMarshaller marsh = new PortableMarshaller();