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 14F6218353 for ; Wed, 18 Nov 2015 13:39:37 +0000 (UTC) Received: (qmail 77176 invoked by uid 500); 18 Nov 2015 13:39:37 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 77016 invoked by uid 500); 18 Nov 2015 13:39:36 -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 76030 invoked by uid 99); 18 Nov 2015 13:39:36 -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, 18 Nov 2015 13:39:36 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 4A0D7E0AA0; Wed, 18 Nov 2015 13:39:36 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vozerov@apache.org To: commits@ignite.apache.org Date: Wed, 18 Nov 2015 13:40:08 -0000 Message-Id: <9b6e428481884fea8dc992561910ad2f@git.apache.org> In-Reply-To: <5e54a9bd0daf4e6fb933fbefbfa05711@git.apache.org> References: <5e54a9bd0daf4e6fb933fbefbfa05711@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [34/50] [abbrv] ignite git commit: IGNITE-1847: Added "field" method to BinaryType and reworked internal metadata handling. http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java index 98b619e..117eece 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java @@ -17,41 +17,26 @@ package org.apache.ignite.internal.processors.cache.portable; -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import javax.cache.Cache; -import javax.cache.CacheException; -import javax.cache.event.CacheEntryEvent; -import javax.cache.event.CacheEntryListenerException; -import javax.cache.event.CacheEntryUpdatedListener; -import javax.cache.event.EventType; -import javax.cache.processor.EntryProcessor; -import javax.cache.processor.MutableEntry; +import org.apache.ignite.IgniteBinary; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.cluster.ClusterTopologyException; -import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; -import org.apache.ignite.IgniteBinary; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryObjectBuilder; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cache.CacheEntryEventSerializableFilter; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.cluster.ClusterTopologyException; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.portable.GridPortableMarshaller; -import org.apache.ignite.internal.portable.PortableContext; -import org.apache.ignite.internal.portable.PortableMetaDataHandler; -import org.apache.ignite.internal.portable.BinaryMetaDataImpl; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.portable.BinaryMetadata; import org.apache.ignite.internal.portable.BinaryObjectImpl; import org.apache.ignite.internal.portable.BinaryObjectOffheapImpl; +import org.apache.ignite.internal.portable.BinaryTypeImpl; +import org.apache.ignite.internal.portable.GridPortableMarshaller; +import org.apache.ignite.internal.portable.PortableContext; +import org.apache.ignite.internal.portable.BinaryMetadataHandler; import org.apache.ignite.internal.portable.PortableUtils; import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.portable.streams.PortableInputStream; @@ -82,51 +67,33 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteClosure; 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.BinaryType; -import org.apache.ignite.binary.BinaryObject; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; import sun.misc.Unsafe; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLASS; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.COL; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.PORTABLE_OBJ; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP_ARR; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID; -import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_ARR; +import javax.cache.Cache; +import javax.cache.CacheException; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.CacheEntryListenerException; +import javax.cache.event.CacheEntryUpdatedListener; +import javax.cache.event.EventType; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.MutableEntry; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; /** * Portable processor implementation. @@ -134,9 +101,6 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_AR public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorImpl implements CacheObjectBinaryProcessor { /** */ - public static final String[] FIELD_TYPE_NAMES; - - /** */ private static final Unsafe UNSAFE = GridUnsafe.unsafe(); /** */ @@ -146,17 +110,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm 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() { private static final long serialVersionUID = 0L; @Override public boolean apply(GridCacheEntryEx e) { - return e.key().value(e.context().cacheObjectContext(), false) instanceof PortableMetaDataKey; + return e.key().value(e.context().cacheObjectContext(), false) instanceof PortableMetadataKey; } }; @@ -174,98 +138,12 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm 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; /** - * - */ - static { - FIELD_TYPE_NAMES = new String[104]; - - FIELD_TYPE_NAMES[BYTE] = "byte"; - FIELD_TYPE_NAMES[SHORT] = "short"; - FIELD_TYPE_NAMES[INT] = "int"; - FIELD_TYPE_NAMES[LONG] = "long"; - FIELD_TYPE_NAMES[BOOLEAN] = "boolean"; - FIELD_TYPE_NAMES[FLOAT] = "float"; - FIELD_TYPE_NAMES[DOUBLE] = "double"; - FIELD_TYPE_NAMES[CHAR] = "char"; - FIELD_TYPE_NAMES[UUID] = "UUID"; - FIELD_TYPE_NAMES[DECIMAL] = "decimal"; - FIELD_TYPE_NAMES[STRING] = "String"; - FIELD_TYPE_NAMES[DATE] = "Date"; - FIELD_TYPE_NAMES[TIMESTAMP] = "Timestamp"; - FIELD_TYPE_NAMES[ENUM] = "Enum"; - FIELD_TYPE_NAMES[OBJ] = "Object"; - FIELD_TYPE_NAMES[PORTABLE_OBJ] = "Object"; - FIELD_TYPE_NAMES[COL] = "Collection"; - FIELD_TYPE_NAMES[MAP] = "Map"; - FIELD_TYPE_NAMES[MAP_ENTRY] = "Entry"; - FIELD_TYPE_NAMES[CLASS] = "Class"; - FIELD_TYPE_NAMES[BYTE_ARR] = "byte[]"; - FIELD_TYPE_NAMES[SHORT_ARR] = "short[]"; - FIELD_TYPE_NAMES[INT_ARR] = "int[]"; - FIELD_TYPE_NAMES[LONG_ARR] = "long[]"; - FIELD_TYPE_NAMES[BOOLEAN_ARR] = "boolean[]"; - FIELD_TYPE_NAMES[FLOAT_ARR] = "float[]"; - FIELD_TYPE_NAMES[DOUBLE_ARR] = "double[]"; - FIELD_TYPE_NAMES[CHAR_ARR] = "char[]"; - FIELD_TYPE_NAMES[UUID_ARR] = "UUID[]"; - FIELD_TYPE_NAMES[DECIMAL_ARR] = "decimal[]"; - FIELD_TYPE_NAMES[STRING_ARR] = "String[]"; - FIELD_TYPE_NAMES[DATE_ARR] = "Date[]"; - FIELD_TYPE_NAMES[TIMESTAMP_ARR] = "Timestamp[]"; - FIELD_TYPE_NAMES[OBJ_ARR] = "Object[]"; - FIELD_TYPE_NAMES[ENUM_ARR] = "Enum[]"; - } - - /** - * @param typeName Field type name. - * @return Field type ID; - */ - @SuppressWarnings("StringEquality") - public static int fieldTypeId(String typeName) { - for (int i = 0; i < FIELD_TYPE_NAMES.length; i++) { - String typeName0 = FIELD_TYPE_NAMES[i]; - - if (typeName.equals(typeName0)) - return i; - } - - throw new IllegalArgumentException("Invalid metadata type name: " + typeName); - } - - /** - * @param typeId Field type ID. - * @return Field type name. - */ - public static String fieldTypeName(int typeId) { - assert typeId >= 0 && typeId < FIELD_TYPE_NAMES.length : typeId; - - String typeName = FIELD_TYPE_NAMES[typeId]; - - assert typeName != null : typeId; - - return typeName; - } - - /** - * @param typeIds Field type IDs. - * @return Field type names. - */ - public static Map fieldTypeNames(Map typeIds) { - Map names = U.newHashMap(typeIds.size()); - - for (Map.Entry e : typeIds.entrySet()) - names.put(e.getKey(), fieldTypeName(e.getValue())); - - return names; - } - - /** * @param ctx Kernal context. */ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { @@ -275,28 +153,31 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm 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, BinaryType newMeta) - throws BinaryObjectException { + BinaryMetadataHandler metaHnd = new BinaryMetadataHandler() { + @Override public void addMeta(int typeId, BinaryType newMeta) throws BinaryObjectException { + assert newMeta != null; + assert newMeta instanceof BinaryTypeImpl; + + BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata(); + if (metaDataCache == null) { - BinaryType oldMeta = metaBuf.get(typeId); + BinaryMetadata oldMeta = metaBuf.get(typeId); - if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta, null)) { + if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) { synchronized (this) { - Map fields = new HashMap<>(); + Map fields = new HashMap<>(); - if (checkMeta(typeId, oldMeta, newMeta, fields)) { - newMeta = new BinaryMetaDataImpl(newMeta.typeName(), - fields, - newMeta.affinityKeyFieldName()); + if (checkMeta(typeId, oldMeta, newMeta0, fields)) { + newMeta0 = new BinaryMetadata(typeId, newMeta0.typeName(), fields, + newMeta0.affinityKeyFieldName()); - metaBuf.put(typeId, newMeta); + metaBuf.put(typeId, newMeta0); } else return; @@ -311,7 +192,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm return; } - CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta); + CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta0.wrap(portableCtx)); } @Override public BinaryType metadata(int typeId) throws BinaryObjectException { @@ -357,7 +238,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm GridCacheQueryManager qryMgr = metaDataCache.context().queries(); - CacheQuery> qry = + CacheQuery> qry = qryMgr.createScanQuery(new MetaDataPredicate(), null, false); qry.keepAll(false); @@ -365,9 +246,9 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm 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; @@ -395,8 +276,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm startLatch.countDown(); - for (Map.Entry e : metaBuf.entrySet()) - addMeta(e.getKey(), e.getValue()); + for (Map.Entry e : metaBuf.entrySet()) + addMeta(e.getKey(), e.getValue().wrap(portableCtx)); metaBuf.clear(); } @@ -413,20 +294,24 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm * @param key Metadata key. * @param newMeta Metadata. */ - private void addClientCacheMetaData(PortableMetaDataKey key, final BinaryType newMeta) { - clientMetaDataCache.compute(key, - new ConcurrentHashMap8.BiFun() { - @Override public BinaryType apply(PortableMetaDataKey key, BinaryType oldMeta) { - BinaryType res; + private void addClientCacheMetaData(PortableMetadataKey key, final BinaryMetadata newMeta) { + int key0 = key.typeId(); + + clientMetaDataCache.compute(key0, + new ConcurrentHashMap8.BiFun() { + @Override public BinaryTypeImpl apply(Integer key, BinaryTypeImpl oldMeta) { + BinaryMetadata res; + + BinaryMetadata oldMeta0 = oldMeta != null ? oldMeta.metadata() : null; try { - res = checkMeta(key.typeId(), oldMeta, newMeta, null) ? newMeta : oldMeta; + res = checkMeta(key, oldMeta0, newMeta, null) ? newMeta : oldMeta0; } catch (BinaryObjectException e) { - res = oldMeta; + res = oldMeta0; } - return res; + return res != null ? res.wrap(portableCtx) : null; } } ); @@ -480,6 +365,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm } /** {@inheritDoc} */ + @SuppressWarnings("unchecked") @Override public Object marshalToPortable(@Nullable Object obj) throws BinaryObjectException { if (obj == null) return null; @@ -564,21 +450,23 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** {@inheritDoc} */ @Override public void updateMetaData(int typeId, String typeName, @Nullable String affKeyFieldName, Map fieldTypeIds) throws BinaryObjectException { - portableCtx.updateMetaData(typeId, - new BinaryMetaDataImpl(typeName, fieldTypeNames(fieldTypeIds), affKeyFieldName)); + portableCtx.updateMetaData(typeId, new BinaryMetadata(typeId, typeName, fieldTypeIds, affKeyFieldName)); } /** {@inheritDoc} */ @Override public void addMeta(final int typeId, final BinaryType newMeta) throws BinaryObjectException { assert newMeta != null; + assert newMeta instanceof BinaryTypeImpl; + + BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata(); - final PortableMetaDataKey key = new PortableMetaDataKey(typeId); + final PortableMetadataKey key = new PortableMetadataKey(typeId); try { - BinaryType oldMeta = metaDataCache.localPeek(key); + BinaryMetadata oldMeta = metaDataCache.localPeek(key); - if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta, null)) { - BinaryObjectException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta)); + if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) { + BinaryObjectException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta0)); if (err != null) throw err; @@ -593,9 +481,12 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm @Nullable @Override public BinaryType metadata(final int typeId) throws BinaryObjectException { try { if (clientNode) - return clientMetaDataCache.get(new PortableMetaDataKey(typeId)); + return clientMetaDataCache.get(typeId); + else { + BinaryMetadata meta = metaDataCache.localPeek(new PortableMetadataKey(typeId)); - return metaDataCache.localPeek(new PortableMetaDataKey(typeId)); + return meta != null ? meta.wrap(portableCtx) : null; + } } catch (CacheException e) { throw new BinaryObjectException(e); @@ -606,17 +497,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm @Override public Map metadata(Collection typeIds) throws BinaryObjectException { try { - Collection keys = new ArrayList<>(typeIds.size()); + Collection keys = new ArrayList<>(typeIds.size()); for (Integer typeId : typeIds) - keys.add(new PortableMetaDataKey(typeId)); + keys.add(new PortableMetadataKey(typeId)); - Map meta = metaDataCache.getAll(keys); + Map meta = metaDataCache.getAll(keys); Map res = U.newHashMap(meta.size()); - for (Map.Entry e : meta.entrySet()) - res.put(e.getKey().typeId(), e.getValue()); + for (Map.Entry e : meta.entrySet()) + res.put(e.getKey().typeId(), e.getValue().wrap(portableCtx)); return res; } @@ -629,17 +520,21 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm @SuppressWarnings("unchecked") @Override public Collection metadata() throws BinaryObjectException { if (clientNode) - return new ArrayList<>(clientMetaDataCache.values()); - - return F.viewReadOnly(metaDataCache.entrySetx(metaPred), - new C1, BinaryType>() { - private static final long serialVersionUID = 0L; - - @Override public BinaryType apply( - Cache.Entry e) { - return e.getValue(); + return F.viewReadOnly(clientMetaDataCache.values(), new IgniteClosure() { + @Override public BinaryType apply(BinaryTypeImpl meta) { + return meta; } }); + else { + return F.viewReadOnly(metaDataCache.entrySetx(metaPred), + new C1, BinaryType>() { + private static final long serialVersionUID = 0L; + + @Override public BinaryType apply(Cache.Entry e) { + return e.getValue().wrap(portableCtx); + } + }); + } } /** {@inheritDoc} */ @@ -839,12 +734,12 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm * @return Whether meta is changed. * @throws org.apache.ignite.binary.BinaryObjectException In case of error. */ - private static boolean checkMeta(int typeId, @Nullable BinaryType oldMeta, - BinaryType newMeta, @Nullable Map fields) throws BinaryObjectException { + private static boolean checkMeta(int typeId, @Nullable BinaryMetadata oldMeta, + BinaryMetadata newMeta, @Nullable Map fields) throws BinaryObjectException { assert newMeta != null; - Map oldFields = oldMeta != null ? ((BinaryMetaDataImpl)oldMeta).fieldsMeta() : null; - Map newFields = ((BinaryMetaDataImpl)newMeta).fieldsMeta(); + Map oldFields = oldMeta != null ? oldMeta.fieldsMap() : null; + Map newFields = newMeta.fieldsMap(); boolean changed = false; @@ -875,17 +770,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm else changed = true; - for (Map.Entry e : newFields.entrySet()) { - String typeName = oldFields != null ? oldFields.get(e.getKey()) : null; + for (Map.Entry e : newFields.entrySet()) { + Integer oldTypeId = oldFields != null ? oldFields.get(e.getKey()) : null; - if (typeName != null) { - if (!typeName.equals(e.getValue())) { + if (oldTypeId != null) { + if (!oldTypeId.equals(e.getValue())) { throw new BinaryObjectException( "Portable field has different types on different clients [" + "typeName=" + newMeta.typeName() + ", fieldName=" + e.getKey() + - ", fieldTypeName1=" + typeName + - ", fieldTypeName2=" + e.getValue() + + ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldTypeId) + + ", fieldTypeName2=" + PortableUtils.fieldTypeName(e.getValue()) + ']' ); } @@ -904,7 +799,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** */ private static class MetaDataProcessor implements - EntryProcessor, Externalizable { + EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -912,7 +807,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm private int typeId; /** */ - private BinaryType newMeta; + private BinaryMetadata newMeta; /** * For {@link Externalizable}. @@ -925,7 +820,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm * @param typeId Type ID. * @param newMeta New metadata. */ - private MetaDataProcessor(int typeId, BinaryType newMeta) { + private MetaDataProcessor(int typeId, BinaryMetadata newMeta) { assert newMeta != null; this.typeId = typeId; @@ -934,16 +829,15 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** {@inheritDoc} */ @Override public BinaryObjectException process( - MutableEntry entry, + MutableEntry entry, Object... args) { try { - BinaryType oldMeta = entry.getValue(); + BinaryMetadata oldMeta = entry.getValue(); - Map fields = new HashMap<>(); + Map fields = new HashMap<>(); if (checkMeta(typeId, oldMeta, newMeta, fields)) { - BinaryType res = new BinaryMetaDataImpl(newMeta.typeName(), - fields, + BinaryMetadata res = new BinaryMetadata(typeId, newMeta.typeName(), fields, newMeta.affinityKeyFieldName()); entry.setValue(res); @@ -967,7 +861,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** {@inheritDoc} */ @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { typeId = in.readInt(); - newMeta = (BinaryType)in.readObject(); + newMeta = (BinaryMetadata)in.readObject(); } /** {@inheritDoc} */ @@ -979,17 +873,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** * */ - 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(); + PortableMetadataKey key = evt.getKey(); - final BinaryType newMeta = evt.getValue(); + final BinaryMetadata newMeta = evt.getValue(); assert newMeta != null : evt; @@ -1012,7 +906,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** {@inheritDoc} */ @Override public boolean evaluate(CacheEntryEvent evt) throws CacheEntryListenerException { - return evt.getKey() instanceof PortableMetaDataKey; + return evt.getKey() instanceof PortableMetadataKey; } /** {@inheritDoc} */ @@ -1030,7 +924,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** {@inheritDoc} */ @Override public boolean apply(Object key, Object val) { - return key instanceof PortableMetaDataKey; + return key instanceof PortableMetadataKey; } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/PortableMetaDataKey.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/PortableMetaDataKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/PortableMetaDataKey.java index 2a98778..f838c82 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/PortableMetaDataKey.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/PortableMetaDataKey.java @@ -27,7 +27,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; /** * Key for portable meta data. */ -class PortableMetaDataKey extends GridCacheUtilityKey implements Externalizable { +class PortableMetadataKey extends GridCacheUtilityKey implements Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -37,14 +37,14 @@ class PortableMetaDataKey extends GridCacheUtilityKey imple /** * For {@link Externalizable}. */ - public PortableMetaDataKey() { + public PortableMetadataKey() { // No-op. } /** * @param typeId Type ID. */ - PortableMetaDataKey(int typeId) { + PortableMetadataKey(int typeId) { this.typeId = typeId; } @@ -66,7 +66,7 @@ class PortableMetaDataKey extends GridCacheUtilityKey imple } /** {@inheritDoc} */ - @Override protected boolean equalsx(PortableMetaDataKey key) { + @Override protected boolean equalsx(PortableMetadataKey key) { return typeId == key.typeId; } @@ -77,6 +77,6 @@ class PortableMetaDataKey extends GridCacheUtilityKey imple /** {@inheritDoc} */ @Override public String toString() { - return S.toString(PortableMetaDataKey.class, this); + return S.toString(PortableMetadataKey.class, this); } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 c513600..05d3515 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 @@ -34,7 +34,7 @@ import org.apache.ignite.events.SwapSpaceEvent; import org.apache.ignite.events.TaskEvent; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.portable.GridPortableMarshaller; -import org.apache.ignite.internal.portable.BinaryMetaDataImpl; +import org.apache.ignite.internal.portable.BinaryMetadata; import org.apache.ignite.internal.portable.BinaryRawReaderEx; import org.apache.ignite.internal.portable.BinaryRawWriterEx; import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl; @@ -68,7 +68,6 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformReaderClosur import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; 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.BinaryType; import org.jetbrains.annotations.Nullable; @@ -391,12 +390,7 @@ public class PlatformContextImpl implements PlatformContext { else { writer.writeBoolean(true); - Map metaFields = ((BinaryMetaDataImpl)meta).fields0(); - - Map fields = U.newHashMap(metaFields.size()); - - for (Map.Entry metaField : metaFields.entrySet()) - fields.put(metaField.getKey(), CacheObjectBinaryProcessorImpl.fieldTypeId(metaField.getValue())); + Map fields = ((BinaryMetadata)meta).fieldsMap(); writer.writeInt(typeId); writer.writeString(meta.typeName()); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 60c0693..d0462e9 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 @@ -22,10 +22,10 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.PlatformConfiguration; import org.apache.ignite.internal.MarshallerContextImpl; +import org.apache.ignite.internal.portable.BinaryNoopMetadataHandler; +import org.apache.ignite.internal.portable.BinaryRawWriterEx; import org.apache.ignite.internal.portable.GridPortableMarshaller; import org.apache.ignite.internal.portable.PortableContext; -import org.apache.ignite.internal.portable.PortableMetaDataHandler; -import org.apache.ignite.internal.portable.BinaryRawWriterEx; import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure; import org.apache.ignite.internal.processors.platform.lifecycle.PlatformLifecycleBean; import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream; @@ -36,11 +36,9 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lifecycle.LifecycleBean; import org.apache.ignite.marshaller.Marshaller; -import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration; import org.apache.ignite.marshaller.portable.PortableMarshaller; +import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration; import org.apache.ignite.platform.dotnet.PlatformDotNetLifecycleBean; -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryType; import java.util.ArrayList; import java.util.Collections; @@ -228,16 +226,7 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur @SuppressWarnings("deprecation") private static GridPortableMarshaller marshaller() { try { - PortableContext ctx = new PortableContext(new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryType meta) - throws BinaryObjectException { - // No-op. - } - - @Override public BinaryType metadata(int typeId) throws BinaryObjectException { - return null; - } - }, new IgniteConfiguration()); + PortableContext ctx = new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration()); PortableMarshaller marsh = new PortableMarshaller(); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index c4deaa0..4d1145d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -1875,7 +1875,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { if (field0 == null) { - field0 = obj.fieldDescriptor(propName); + field0 = obj.type().field(propName); assert field0 != null; http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java index 4fa80b4..14fc6f3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java @@ -17,15 +17,14 @@ package org.apache.ignite.internal.portable; +import org.apache.ignite.binary.BinaryField; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.IgniteUtils; 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.BinaryField; -import org.apache.ignite.binary.BinaryType; -import org.apache.ignite.binary.BinaryObject; -import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import java.math.BigDecimal; @@ -38,17 +37,6 @@ import java.util.UUID; * Contains tests for portable object fields. */ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTest { - /** Dummy metadata handler. */ - protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryType meta) { - // No-op. - } - - @Override public BinaryType metadata(int typeId) { - return null; - } - }; - /** Marshaller. */ protected PortableMarshaller dfltMarsh; @@ -59,7 +47,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes * @throws Exception If failed. */ protected static PortableMarshaller createMarshaller() throws Exception { - PortableContext ctx = new PortableContext(META_HND, new IgniteConfiguration()); + PortableContext ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration()); PortableMarshaller marsh = new PortableMarshaller(); @@ -485,7 +473,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes BinaryObjectEx portObj = toPortable(marsh, obj); - BinaryField field = portObj.fieldDescriptor(fieldName); + BinaryField field = portObj.type().field(fieldName); return new TestContext(obj, portObj, field); } @@ -508,7 +496,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes assert portObj != null; - BinaryField field = portObj.fieldDescriptor(fieldName); + BinaryField field = portObj.type().field(fieldName); return new TestContext(obj, portObj, field); } http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 e62d12e..11b54ae 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 @@ -998,7 +998,7 @@ public class GridBinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstrac BinaryType metadata = portables().metadata(c.getClass()); - assertTrue(metadata.fields().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField", + assertTrue(metadata.fieldNames().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField", "colField", "mapField", "enumField", "enumArrField"))); assertEquals("int", metadata.fieldTypeName("intField")); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 ccebd73..a74315b 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 @@ -789,7 +789,7 @@ public class GridBinaryObjectBuilderSelfTest extends GridCommonAbstractTest { assertEquals("MetaTest", meta.typeName()); - Collection fields = meta.fields(); + Collection fields = meta.fieldNames(); assertEquals(2, fields.size()); @@ -812,7 +812,7 @@ public class GridBinaryObjectBuilderSelfTest extends GridCommonAbstractTest { assertEquals("MetaTest", meta.typeName()); - fields = meta.fields(); + fields = meta.fieldNames(); assertEquals(3, fields.size()); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 ee91167..747f8ea 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 @@ -17,38 +17,27 @@ package org.apache.ignite.internal.portable; -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.Arrays; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.binary.Binarylizable; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.MarshallerContextAdapter; 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.BinaryType; -import org.apache.ignite.binary.BinaryReader; -import org.apache.ignite.binary.BinaryWriter; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Arrays; + /** * */ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstractTest { - /** */ - protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryType meta) { - // No-op. - } - - @Override public BinaryType metadata(int typeId) { - return null; - } - }; - /** * @throws Exception If failed. */ @@ -56,7 +45,7 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac PortableMarshaller marsh = new PortableMarshaller(); marsh.setContext(new MarshallerContextWithNoStorage()); - PortableContext context = new PortableContext(META_HND, new IgniteConfiguration()); + PortableContext context = new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration()); IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", context); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 b5a24b1..d0a5709 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 @@ -82,17 +82,6 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest { /** */ protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class); - /** */ - protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryType meta) { - // No-op. - } - - @Override public BinaryType metadata(int typeId) { - return null; - } - }; - /** * @throws Exception If failed. */ @@ -2414,7 +2403,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest { protected PortableContext initPortableContext(PortableMarshaller marsh) throws IgniteCheckedException { IgniteConfiguration iCfg = new IgniteConfiguration(); - PortableContext ctx = new PortableContext(META_HND, iCfg); + PortableContext ctx = new PortableContext(BinaryNoopMetadataHandler.instance(), iCfg); marsh.setContext(new MarshallerContextTestImpl(null)); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 c0e2563..2a367a8 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 @@ -93,7 +93,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { switch (meta.typeName()) { case "TestObject1": - fields = meta.fields(); + fields = meta.fieldNames(); assertEquals(7, fields.size()); @@ -116,7 +116,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { break; case "TestObject2": - fields = meta.fields(); + fields = meta.fieldNames(); assertEquals(7, fields.size()); @@ -165,7 +165,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { assertEquals("TestObject1", meta.typeName()); - Collection fields = meta.fields(); + Collection fields = meta.fieldNames(); assertEquals(7, fields.size()); @@ -198,7 +198,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { assertEquals("TestObject2", meta.typeName()); - Collection fields = meta.fields(); + Collection fields = meta.fieldNames(); assertEquals(7, fields.size()); @@ -235,7 +235,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { assertEquals("TestObject2", meta.typeName()); - Collection fields = meta.fields(); + Collection fields = meta.fieldNames(); assertEquals(9, fields.size()); @@ -284,7 +284,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest { assertEquals("TestObject1", meta.typeName()); - Collection fields = meta.fields(); + Collection fields = meta.fieldNames(); assertEquals(7, fields.size()); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 7522bf9..52af867 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 @@ -17,37 +17,26 @@ package org.apache.ignite.internal.portable; -import java.util.Arrays; -import java.util.Map; +import org.apache.ignite.binary.BinaryIdMapper; +import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; 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.BinaryIdMapper; -import org.apache.ignite.binary.BinaryType; -import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.util.Arrays; +import java.util.Map; + /** * Wildcards test. */ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest { - /** */ - private static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryType meta) { - // No-op. - } - - @Override public BinaryType metadata(int typeId) { - return null; - } - }; - /** * @return Portable context. */ private PortableContext portableContext() { - return new PortableContext(META_HND, new IgniteConfiguration()); + return new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration()); } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java index db3c821..9225b97 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/PortableCompactOffsetsAbstractSelfTest.java @@ -17,9 +17,7 @@ package org.apache.ignite.internal.portable; -import java.util.Arrays; import org.apache.ignite.binary.BinaryField; -import org.apache.ignite.binary.BinaryType; import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.IgniteUtils; @@ -27,6 +25,8 @@ import org.apache.ignite.marshaller.MarshallerContextTestImpl; import org.apache.ignite.marshaller.portable.PortableMarshaller; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.util.Arrays; + /** * Contains tests for compact offsets. */ @@ -37,17 +37,6 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA /** 2 pow 16. */ private static int POW_16 = 1 << 16; - /** Dummy metadata handler. */ - protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryType meta) { - // No-op. - } - - @Override public BinaryType metadata(int typeId) { - return null; - } - }; - /** Marshaller. */ protected PortableMarshaller marsh; @@ -58,7 +47,7 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA @Override protected void beforeTest() throws Exception { super.beforeTest(); - ctx = new PortableContext(META_HND, new IgniteConfiguration()); + ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration()); marsh = new PortableMarshaller(); @@ -139,8 +128,8 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA assert obj.field2 == field2; // 2. Test fields API. - BinaryField field1Desc = portObj.fieldDescriptor("field1"); - BinaryField field2Desc = portObj.fieldDescriptor("field2"); + BinaryField field1Desc = portObj.type().field("field1"); + BinaryField field2Desc = portObj.type().field("field2"); assert field1Desc.exists(portObj); assert field2Desc.exists(portObj); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java new file mode 100644 index 0000000..e49ebf3 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestCachingMetadataHandler.java @@ -0,0 +1,45 @@ +/* + * 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.portable; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryType; + +import java.util.concurrent.ConcurrentHashMap; + +/** + * Test metadata handler. + */ +public class TestCachingMetadataHandler implements BinaryMetadataHandler { + /** Cached metadatas. */ + private final ConcurrentHashMap metas = new ConcurrentHashMap<>(); + + /** {@inheritDoc} */ + @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException { + BinaryType otherType = metas.put(typeId, meta); + + if (otherType != null) + throw new IllegalStateException("Metadata replacement is not allowed in " + + TestCachingMetadataHandler.class.getSimpleName() + '.'); + } + + /** {@inheritDoc} */ + @Override public BinaryType metadata(int typeId) throws BinaryObjectException { + return metas.get(typeId); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 a33eb7b..d19c1ce 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 @@ -169,7 +169,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo assertNull(meta.affinityKeyFieldName()); - assertEquals(10, meta.fields().size()); + assertEquals(10, meta.fieldNames().size()); } assertEquals(allTypes.size(), names.size()); @@ -255,7 +255,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo assertNull(meta.affinityKeyFieldName()); - assertEquals(1, meta.fields().size()); + assertEquals(1, meta.fieldNames().size()); } assertEquals(1000, names.size()); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 b20adb8..10c06a7 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 @@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.portable; import java.util.Arrays; import java.util.Collection; import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheKeyConfiguration; import org.apache.ignite.cache.CacheMode; @@ -29,9 +28,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; 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.BinaryType; -import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryTypeConfiguration; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -127,7 +124,7 @@ public class GridCacheClientNodeBinaryObjectMetadataTest extends GridCacheAbstra for (BinaryType m2 : meta1) { if (m1.typeName().equals(m2.typeName())) { assertEquals(m1.affinityKeyFieldName(), m2.affinityKeyFieldName()); - assertEquals(m1.fields(), m2.fields()); + assertEquals(m1.fieldNames(), m2.fieldNames()); found = true; http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 d277801..2e868da 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 @@ -19,15 +19,13 @@ package org.apache.ignite.internal.processors.cache.portable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.portable.BinaryNoopMetadataHandler; import org.apache.ignite.internal.portable.PortableContext; -import org.apache.ignite.internal.portable.PortableMetaDataHandler; import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest; import org.apache.ignite.internal.util.IgniteUtils; 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.BinaryType; /** * @@ -39,15 +37,7 @@ public class GridPortableCacheEntryMemorySizeSelfTest extends GridCacheEntryMemo marsh.setContext(new MarshallerContextTestImpl(null)); - PortableContext pCtx = new PortableContext(new PortableMetaDataHandler() { - @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException { - // No-op - } - - @Override public BinaryType metadata(int typeId) throws BinaryObjectException { - return null; - } - }, new IgniteConfiguration()); + PortableContext pCtx = new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration()); IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", pCtx); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java index 77c7e3a..8eba80b 100644 --- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java +++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java @@ -94,7 +94,7 @@ public class PlatformComputeBinarizableArgTask extends ComputeTaskAdapter