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 74B1318493 for ; Tue, 10 Nov 2015 12:21:42 +0000 (UTC) Received: (qmail 72320 invoked by uid 500); 10 Nov 2015 12:21:42 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 72223 invoked by uid 500); 10 Nov 2015 12:21:42 -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 72020 invoked by uid 99); 10 Nov 2015 12:21:42 -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, 10 Nov 2015 12:21:42 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 02E32E56E3; Tue, 10 Nov 2015 12:21:42 +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: Tue, 10 Nov 2015 12:21:49 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [9/9] ignite git commit: IGNITE-1847: Refactoring metadata handlers. IGNITE-1847: Refactoring metadata handlers. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b2fd7bfd Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b2fd7bfd Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b2fd7bfd Branch: refs/heads/ignite-1847 Commit: b2fd7bfd41817693cdb10fc80c05e3365b177f12 Parents: fc49be0 Author: vozerov-gridgain Authored: Tue Nov 10 15:22:21 2015 +0300 Committer: vozerov-gridgain Committed: Tue Nov 10 15:22:21 2015 +0300 ---------------------------------------------------------------------- .../portable/BinaryMetaDataCollector.java | 254 ------------------- .../internal/portable/BinaryMetaDataImpl.java | 152 ----------- .../internal/portable/BinaryMetadata.java | 152 +++++++++++ .../portable/BinaryMetadataCollector2.java | 254 +++++++++++++++++++ .../portable/BinaryMetadataHandler2.java | 44 ++++ .../portable/BinaryNoopMetadataHandler.java | 53 ++++ .../internal/portable/BinaryTypeImpl.java | 6 +- .../portable/PortableClassDescriptor.java | 4 +- .../internal/portable/PortableContext.java | 10 +- .../portable/PortableMetaDataHandler.java | 44 ---- .../CacheObjectBinaryProcessorImpl.java | 72 +++--- .../platform/PlatformContextImpl.java | 4 +- .../PlatformDotNetConfigurationClosure.java | 19 +- .../portable/BinaryFieldsAbstractSelfTest.java | 2 +- ...idPortableMarshallerCtxDisabledSelfTest.java | 2 +- .../GridPortableMarshallerSelfTest.java | 2 +- .../portable/GridPortableWildcardsSelfTest.java | 2 +- .../PortableCompactOffsetsAbstractSelfTest.java | 2 +- .../portable/TestCachingMetadataHandler.java | 45 ++++ .../internal/portable/TestMetadataHandler.java | 45 ---- .../portable/TestNoopMetadataHandler.java | 36 --- ...ridPortableCacheEntryMemorySizeSelfTest.java | 4 +- 22 files changed, 607 insertions(+), 601 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java deleted file mode 100644 index b9ca926..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataCollector.java +++ /dev/null @@ -1,254 +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.portable; - -import java.lang.reflect.InvocationHandler; -import java.lang.reflect.Method; -import java.lang.reflect.Proxy; -import java.math.BigDecimal; -import java.sql.Timestamp; -import java.util.Collection; -import java.util.Date; -import java.util.HashMap; -import java.util.Map; -import java.util.UUID; -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryRawWriter; -import org.apache.ignite.binary.BinaryWriter; -import org.jetbrains.annotations.Nullable; - -/** - * Writer for meta data collection. - */ -class BinaryMetaDataCollector implements BinaryWriter { - /** */ - private final Map meta = new HashMap<>(); - - /** */ - private final String typeName; - - /** - * @param typeName Type name. - */ - BinaryMetaDataCollector(String typeName) { - this.typeName = typeName; - } - - /** - * @return Field meta data. - */ - Map meta() { - return meta; - } - - /** {@inheritDoc} */ - @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.BYTE); - } - - /** {@inheritDoc} */ - @Override public void writeShort(String fieldName, short val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.SHORT); - } - - /** {@inheritDoc} */ - @Override public void writeInt(String fieldName, int val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.INT); - } - - /** {@inheritDoc} */ - @Override public void writeLong(String fieldName, long val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.LONG); - } - - /** {@inheritDoc} */ - @Override public void writeFloat(String fieldName, float val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.FLOAT); - } - - /** {@inheritDoc} */ - @Override public void writeDouble(String fieldName, double val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.DOUBLE); - } - - /** {@inheritDoc} */ - @Override public void writeChar(String fieldName, char val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.CHAR); - } - - /** {@inheritDoc} */ - @Override public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.BOOLEAN); - } - - /** {@inheritDoc} */ - @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.DECIMAL); - } - - /** {@inheritDoc} */ - @Override public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.STRING); - } - - /** {@inheritDoc} */ - @Override public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.UUID); - } - - /** {@inheritDoc} */ - @Override public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.DATE); - } - - /** {@inheritDoc} */ - @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP); - } - - /** {@inheritDoc} */ - @Override public > void writeEnum(String fieldName, T val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.ENUM); - } - - /** {@inheritDoc} */ - @Override public > void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.ENUM_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.OBJECT); - } - - /** {@inheritDoc} */ - @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.BYTE_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.SHORT_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.INT_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.LONG_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.FLOAT_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.DOUBLE_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.CHAR_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.BOOLEAN_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.DECIMAL_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.STRING_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.UUID_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.DATE_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.OBJECT_ARR); - } - - /** {@inheritDoc} */ - @Override public void writeCollection(String fieldName, @Nullable Collection col) - throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.COL); - } - - /** {@inheritDoc} */ - @Override public void writeMap(String fieldName, @Nullable Map map) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.MAP); - } - - /** {@inheritDoc} */ - @Override public BinaryRawWriter rawWriter() { - return (BinaryRawWriter)Proxy.newProxyInstance(getClass().getClassLoader(), - new Class[] { BinaryRawWriterEx.class }, - new InvocationHandler() { - @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable { - return null; - } - }); - } - - /** - * @param name Field name. - * @param mode Field mode. - * @throws BinaryObjectException In case of error. - */ - private void add(String name, PortableClassDescriptor.Mode mode) throws BinaryObjectException { - assert name != null; - - int fieldTypeId = mode.typeId(); - - Integer oldFieldTypeId = meta.put(name, fieldTypeId); - - if (oldFieldTypeId != null && !oldFieldTypeId.equals(fieldTypeId)) { - throw new BinaryObjectException( - "Field is written twice with different types [" + - "typeName=" + typeName + - ", fieldName=" + name + - ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldTypeId) + - ", fieldTypeName2=" + PortableUtils.fieldTypeName(fieldTypeId) + - ']' - ); - } - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java deleted file mode 100644 index a09e14a..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java +++ /dev/null @@ -1,152 +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.portable; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import org.apache.ignite.internal.util.tostring.GridToStringInclude; -import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.jetbrains.annotations.Nullable; - -/** - * Portable metadata which is passed over a wire. - */ -public class BinaryMetaDataImpl implements Externalizable { - /** */ - private static final long serialVersionUID = 0L; - - /** Type ID. */ - private int typeId; - - /** Type name. */ - private String typeName; - - /** Recorded object fields. */ - @GridToStringInclude - private Map fields; - - /** Affinity key field name. */ - private String affKeyFieldName; - - /** - * For {@link Externalizable}. - */ - public BinaryMetaDataImpl() { - // No-op. - } - - /** - * Constructor. - * - * @param typeId Type ID. - * @param typeName Type name. - * @param fields Fields map. - * @param affKeyFieldName Affinity key field name. - */ - public BinaryMetaDataImpl(int typeId, String typeName, @Nullable Map fields, - @Nullable String affKeyFieldName) { - assert typeName != null; - - this.typeId = typeId; - this.typeName = typeName; - this.fields = fields; - this.affKeyFieldName = affKeyFieldName; - } - - /** - * @return Type ID. - */ - public int typeId() { - return typeId; - } - - /** - * @return Type name. - */ - public String typeName() { - return typeName; - } - - /** - * @return Fields. - */ - public Collection fields() { - return fields != null ? fields.keySet() : Collections.emptyList(); - } - - /** - * @return Fields. - */ - public Map fieldsMap() { - return fields != null ? fields : Collections.emptyMap(); - } - - /** - * @param fieldName Field name. - * @return Field type name. - */ - @Nullable public String fieldTypeName(String fieldName) { - Integer typeId = fields != null ? fields.get(fieldName) : null; - - return typeId != null ? PortableUtils.fieldTypeName(typeId) : null; - } - - /** - * @return Affinity key field name. - */ - @Nullable public String affinityKeyFieldName() { - return affKeyFieldName; - } - - /** - * Wrap metadata into binary type. - * - * @param ctx Portable context. - * @return Binary type. - */ - public BinaryTypeImpl wrap(PortableContext ctx) { - return new BinaryTypeImpl(ctx, this); - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeInt(typeId); - U.writeString(out, typeName); - U.writeMap(out, fields); - U.writeString(out, affKeyFieldName); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - typeId = in.readInt(); - typeName = U.readString(in); - fields = U.readMap(in); - affKeyFieldName = U.readString(in); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(BinaryMetaDataImpl.class, this); - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java new file mode 100644 index 0000000..fe88d11 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadata.java @@ -0,0 +1,152 @@ +/* + * 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 java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +/** + * Portable metadata which is passed over a wire. + */ +public class BinaryMetadata implements Externalizable { + /** */ + private static final long serialVersionUID = 0L; + + /** Type ID. */ + private int typeId; + + /** Type name. */ + private String typeName; + + /** Recorded object fields. */ + @GridToStringInclude + private Map fields; + + /** Affinity key field name. */ + private String affKeyFieldName; + + /** + * For {@link Externalizable}. + */ + public BinaryMetadata() { + // No-op. + } + + /** + * Constructor. + * + * @param typeId Type ID. + * @param typeName Type name. + * @param fields Fields map. + * @param affKeyFieldName Affinity key field name. + */ + public BinaryMetadata(int typeId, String typeName, @Nullable Map fields, + @Nullable String affKeyFieldName) { + assert typeName != null; + + this.typeId = typeId; + this.typeName = typeName; + this.fields = fields; + this.affKeyFieldName = affKeyFieldName; + } + + /** + * @return Type ID. + */ + public int typeId() { + return typeId; + } + + /** + * @return Type name. + */ + public String typeName() { + return typeName; + } + + /** + * @return Fields. + */ + public Collection fields() { + return fields != null ? fields.keySet() : Collections.emptyList(); + } + + /** + * @return Fields. + */ + public Map fieldsMap() { + return fields != null ? fields : Collections.emptyMap(); + } + + /** + * @param fieldName Field name. + * @return Field type name. + */ + @Nullable public String fieldTypeName(String fieldName) { + Integer typeId = fields != null ? fields.get(fieldName) : null; + + return typeId != null ? PortableUtils.fieldTypeName(typeId) : null; + } + + /** + * @return Affinity key field name. + */ + @Nullable public String affinityKeyFieldName() { + return affKeyFieldName; + } + + /** + * Wrap metadata into binary type. + * + * @param ctx Portable context. + * @return Binary type. + */ + public BinaryTypeImpl wrap(PortableContext ctx) { + return new BinaryTypeImpl(ctx, this); + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeInt(typeId); + U.writeString(out, typeName); + U.writeMap(out, fields); + U.writeString(out, affKeyFieldName); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + typeId = in.readInt(); + typeName = U.readString(in); + fields = U.readMap(in); + affKeyFieldName = U.readString(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(BinaryMetadata.class, this); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector2.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector2.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector2.java new file mode 100644 index 0000000..701c619 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataCollector2.java @@ -0,0 +1,254 @@ +/* + * 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 java.lang.reflect.InvocationHandler; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.binary.BinaryWriter; +import org.jetbrains.annotations.Nullable; + +/** + * Writer for meta data collection. + */ +class BinaryMetadataCollector2 implements BinaryWriter { + /** */ + private final Map meta = new HashMap<>(); + + /** */ + private final String typeName; + + /** + * @param typeName Type name. + */ + BinaryMetadataCollector2(String typeName) { + this.typeName = typeName; + } + + /** + * @return Field meta data. + */ + Map meta() { + return meta; + } + + /** {@inheritDoc} */ + @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.BYTE); + } + + /** {@inheritDoc} */ + @Override public void writeShort(String fieldName, short val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.SHORT); + } + + /** {@inheritDoc} */ + @Override public void writeInt(String fieldName, int val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.INT); + } + + /** {@inheritDoc} */ + @Override public void writeLong(String fieldName, long val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.LONG); + } + + /** {@inheritDoc} */ + @Override public void writeFloat(String fieldName, float val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.FLOAT); + } + + /** {@inheritDoc} */ + @Override public void writeDouble(String fieldName, double val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.DOUBLE); + } + + /** {@inheritDoc} */ + @Override public void writeChar(String fieldName, char val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.CHAR); + } + + /** {@inheritDoc} */ + @Override public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.BOOLEAN); + } + + /** {@inheritDoc} */ + @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.DECIMAL); + } + + /** {@inheritDoc} */ + @Override public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.STRING); + } + + /** {@inheritDoc} */ + @Override public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.UUID); + } + + /** {@inheritDoc} */ + @Override public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.DATE); + } + + /** {@inheritDoc} */ + @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP); + } + + /** {@inheritDoc} */ + @Override public > void writeEnum(String fieldName, T val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.ENUM); + } + + /** {@inheritDoc} */ + @Override public > void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.ENUM_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.OBJECT); + } + + /** {@inheritDoc} */ + @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.BYTE_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.SHORT_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.INT_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.LONG_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.FLOAT_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.DOUBLE_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.CHAR_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.BOOLEAN_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.DECIMAL_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.STRING_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.UUID_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.DATE_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.OBJECT_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeCollection(String fieldName, @Nullable Collection col) + throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.COL); + } + + /** {@inheritDoc} */ + @Override public void writeMap(String fieldName, @Nullable Map map) throws BinaryObjectException { + add(fieldName, PortableClassDescriptor.Mode.MAP); + } + + /** {@inheritDoc} */ + @Override public BinaryRawWriter rawWriter() { + return (BinaryRawWriter)Proxy.newProxyInstance(getClass().getClassLoader(), + new Class[] { BinaryRawWriterEx.class }, + new InvocationHandler() { + @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable { + return null; + } + }); + } + + /** + * @param name Field name. + * @param mode Field mode. + * @throws BinaryObjectException In case of error. + */ + private void add(String name, PortableClassDescriptor.Mode mode) throws BinaryObjectException { + assert name != null; + + int fieldTypeId = mode.typeId(); + + Integer oldFieldTypeId = meta.put(name, fieldTypeId); + + if (oldFieldTypeId != null && !oldFieldTypeId.equals(fieldTypeId)) { + throw new BinaryObjectException( + "Field is written twice with different types [" + + "typeName=" + typeName + + ", fieldName=" + name + + ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldTypeId) + + ", fieldTypeName2=" + PortableUtils.fieldTypeName(fieldTypeId) + + ']' + ); + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler2.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler2.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler2.java new file mode 100644 index 0000000..6e078a1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler2.java @@ -0,0 +1,44 @@ +/* + * 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; + +/** + * Portable meta data handler. + */ +public interface BinaryMetadataHandler2 { + /** + * Adds meta data. + * + * @param typeId Type ID. + * @param meta Meta data. + * @throws org.apache.ignite.binary.BinaryObjectException In case of error. + */ + public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException; + + /** + * Gets meta data for provided type ID. + * + * @param typeId Type ID. + * @return Meta data. + * @throws org.apache.ignite.binary.BinaryObjectException In case of error. + */ + public BinaryType metadata(int typeId) throws BinaryObjectException; +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryNoopMetadataHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryNoopMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryNoopMetadataHandler.java new file mode 100644 index 0000000..fd0c912 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryNoopMetadataHandler.java @@ -0,0 +1,53 @@ +/* + * 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; + +/** + * No-op metadata handler. + */ +public class BinaryNoopMetadataHandler implements BinaryMetadataHandler2 { + /** Cached singleton instance. */ + private static final BinaryNoopMetadataHandler INSTANCE = new BinaryNoopMetadataHandler(); + + /** + * @return Instance. + */ + public static BinaryNoopMetadataHandler instance() { + return INSTANCE; + } + + /** + * Private constructor. + */ + private BinaryNoopMetadataHandler() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public BinaryType metadata(int typeId) throws BinaryObjectException { + return null; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java index 9effdce..40b6252 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java @@ -29,7 +29,7 @@ public class BinaryTypeImpl implements BinaryType { private final PortableContext ctx; /** Type metadata. */ - private final BinaryMetaDataImpl meta; + private final BinaryMetadata meta; /** * Constructor. @@ -37,7 +37,7 @@ public class BinaryTypeImpl implements BinaryType { * @param ctx Portable context. * @param meta Type metadata. */ - public BinaryTypeImpl(PortableContext ctx, BinaryMetaDataImpl meta) { + public BinaryTypeImpl(PortableContext ctx, BinaryMetadata meta) { this.ctx = ctx; this.meta = meta; } @@ -69,7 +69,7 @@ public class BinaryTypeImpl implements BinaryType { /** * @return Metadata. */ - public BinaryMetaDataImpl metadata() { + public BinaryMetadata metadata() { return meta; } } http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java index 422506a..6fdc6b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java @@ -544,9 +544,9 @@ public class PortableClassDescriptor { writer.popSchema(); } - if (obj.getClass() != BinaryMetaDataImpl.class + if (obj.getClass() != BinaryMetadata.class && ctx.isMetaDataChanged(typeId, writer.metaDataHashSum())) { - BinaryMetaDataCollector metaCollector = new BinaryMetaDataCollector(typeName); + BinaryMetadataCollector2 metaCollector = new BinaryMetadataCollector2(typeName); if (serializer != null) serializer.writeBinary(obj, metaCollector); http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java index d54e9b4..8af0630 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java @@ -134,7 +134,7 @@ public class PortableContext implements Externalizable { private final Map typeMappers = new ConcurrentHashMap8<>(0); /** */ - private PortableMetaDataHandler metaHnd; + private BinaryMetadataHandler2 metaHnd; /** */ private MarshallerContext marshCtx; @@ -165,7 +165,7 @@ public class PortableContext implements Externalizable { * @param metaHnd Meta data handler. * @param igniteCfg Ignite configuration. */ - public PortableContext(PortableMetaDataHandler metaHnd, IgniteConfiguration igniteCfg) { + public PortableContext(BinaryMetadataHandler2 metaHnd, IgniteConfiguration igniteCfg) { assert metaHnd != null; assert igniteCfg != null; @@ -777,7 +777,7 @@ public class PortableContext implements Externalizable { descByCls.put(cls, desc); } - metaHnd.addMeta(id, new BinaryMetaDataImpl(id, typeName, fieldsMeta, affKeyFieldName).wrap(this)); + metaHnd.addMeta(id, new BinaryMetadata(id, typeName, fieldsMeta, affKeyFieldName).wrap(this)); } /** @@ -832,7 +832,7 @@ public class PortableContext implements Externalizable { * @throws org.apache.ignite.binary.BinaryObjectException In case of error. */ public void updateMetaData(int typeId, String typeName, Map fields) throws BinaryObjectException { - updateMetaData(typeId, new BinaryMetaDataImpl(typeId, typeName, fields, null)); + updateMetaData(typeId, new BinaryMetadata(typeId, typeName, fields, null)); } /** @@ -840,7 +840,7 @@ public class PortableContext implements Externalizable { * @param meta Meta data. * @throws org.apache.ignite.binary.BinaryObjectException In case of error. */ - public void updateMetaData(int typeId, BinaryMetaDataImpl meta) throws BinaryObjectException { + public void updateMetaData(int typeId, BinaryMetadata meta) throws BinaryObjectException { metaHnd.addMeta(typeId, meta.wrap(this)); } http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java deleted file mode 100644 index 8b2eef2..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java +++ /dev/null @@ -1,44 +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.portable; - -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryType; - -/** - * Portable meta data handler. - */ -public interface PortableMetaDataHandler { - /** - * Adds meta data. - * - * @param typeId Type ID. - * @param meta Meta data. - * @throws org.apache.ignite.binary.BinaryObjectException In case of error. - */ - public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException; - - /** - * Gets meta data for provided type ID. - * - * @param typeId Type ID. - * @return Meta data. - * @throws org.apache.ignite.binary.BinaryObjectException In case of error. - */ - public BinaryType metadata(int typeId) throws BinaryObjectException; -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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 25ffbdf..f8634b7 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 @@ -30,13 +30,13 @@ import org.apache.ignite.cluster.ClusterTopologyException; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; -import org.apache.ignite.internal.portable.BinaryMetaDataImpl; +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.PortableMetaDataHandler; +import org.apache.ignite.internal.portable.BinaryMetadataHandler2; import org.apache.ignite.internal.portable.PortableUtils; import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.portable.streams.PortableInputStream; @@ -110,7 +110,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm private final boolean clientNode; /** */ - private volatile IgniteCacheProxy metaDataCache; + private volatile IgniteCacheProxy metaDataCache; /** */ private final ConcurrentHashMap8 clientMetaDataCache; @@ -138,7 +138,7 @@ 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; @@ -159,22 +159,22 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** {@inheritDoc} */ @Override public void start() throws IgniteCheckedException { if (marsh instanceof PortableMarshaller) { - PortableMetaDataHandler metaHnd = new PortableMetaDataHandler() { + BinaryMetadataHandler2 metaHnd = new BinaryMetadataHandler2() { @Override public void addMeta(int typeId, BinaryType newMeta) throws BinaryObjectException { assert newMeta != null; assert newMeta instanceof BinaryTypeImpl; - BinaryMetaDataImpl newMeta0 = ((BinaryTypeImpl)newMeta).metadata(); + BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata(); if (metaDataCache == null) { - BinaryMetaDataImpl oldMeta = metaBuf.get(typeId); + BinaryMetadata oldMeta = metaBuf.get(typeId); if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) { synchronized (this) { Map fields = new HashMap<>(); if (checkMeta(typeId, oldMeta, newMeta0, fields)) { - newMeta0 = new BinaryMetaDataImpl(typeId, newMeta0.typeName(), fields, + newMeta0 = new BinaryMetadata(typeId, newMeta0.typeName(), fields, newMeta0.affinityKeyFieldName()); metaBuf.put(typeId, newMeta0); @@ -238,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); @@ -246,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; @@ -276,7 +276,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm startLatch.countDown(); - for (Map.Entry e : metaBuf.entrySet()) + for (Map.Entry e : metaBuf.entrySet()) addMeta(e.getKey(), e.getValue().wrap(portableCtx)); metaBuf.clear(); @@ -294,15 +294,15 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm * @param key Metadata key. * @param newMeta Metadata. */ - private void addClientCacheMetaData(PortableMetaDataKey key, final BinaryMetaDataImpl newMeta) { + 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) { - BinaryMetaDataImpl res; + BinaryMetadata res; - BinaryMetaDataImpl oldMeta0 = oldMeta != null ? oldMeta.metadata() : null; + BinaryMetadata oldMeta0 = oldMeta != null ? oldMeta.metadata() : null; try { res = checkMeta(key, oldMeta0, newMeta, null) ? newMeta : oldMeta0; @@ -450,7 +450,7 @@ 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(typeId, typeName, fieldTypeIds, affKeyFieldName)); + portableCtx.updateMetaData(typeId, new BinaryMetadata(typeId, typeName, fieldTypeIds, affKeyFieldName)); } /** {@inheritDoc} */ @@ -458,12 +458,12 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm assert newMeta != null; assert newMeta instanceof BinaryTypeImpl; - BinaryMetaDataImpl newMeta0 = ((BinaryTypeImpl)newMeta).metadata(); + BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata(); final PortableMetaDataKey key = new PortableMetaDataKey(typeId); try { - BinaryMetaDataImpl oldMeta = metaDataCache.localPeek(key); + BinaryMetadata oldMeta = metaDataCache.localPeek(key); if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta0, null)) { BinaryObjectException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta0)); @@ -483,7 +483,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm if (clientNode) return clientMetaDataCache.get(typeId); else { - BinaryMetaDataImpl meta = metaDataCache.localPeek(new PortableMetaDataKey(typeId)); + BinaryMetadata meta = metaDataCache.localPeek(new PortableMetaDataKey(typeId)); return meta != null ? meta.wrap(portableCtx) : null; } @@ -502,11 +502,11 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm 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()); - for (Map.Entry e : meta.entrySet()) + for (Map.Entry e : meta.entrySet()) res.put(e.getKey().typeId(), e.getValue().wrap(portableCtx)); return res; @@ -527,10 +527,10 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm }); else { return F.viewReadOnly(metaDataCache.entrySetx(metaPred), - new C1, BinaryType>() { + new C1, BinaryType>() { private static final long serialVersionUID = 0L; - @Override public BinaryType apply(Cache.Entry e) { + @Override public BinaryType apply(Cache.Entry e) { return e.getValue().wrap(portableCtx); } }); @@ -734,8 +734,8 @@ 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 BinaryMetaDataImpl oldMeta, - BinaryMetaDataImpl 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 ? oldMeta.fieldsMap() : null; @@ -799,7 +799,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** */ private static class MetaDataProcessor implements - EntryProcessor, Externalizable { + EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -807,7 +807,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm private int typeId; /** */ - private BinaryMetaDataImpl newMeta; + private BinaryMetadata newMeta; /** * For {@link Externalizable}. @@ -820,7 +820,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm * @param typeId Type ID. * @param newMeta New metadata. */ - private MetaDataProcessor(int typeId, BinaryMetaDataImpl newMeta) { + private MetaDataProcessor(int typeId, BinaryMetadata newMeta) { assert newMeta != null; this.typeId = typeId; @@ -829,15 +829,15 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** {@inheritDoc} */ @Override public BinaryObjectException process( - MutableEntry entry, + MutableEntry entry, Object... args) { try { - BinaryMetaDataImpl oldMeta = entry.getValue(); + BinaryMetadata oldMeta = entry.getValue(); Map fields = new HashMap<>(); if (checkMeta(typeId, oldMeta, newMeta, fields)) { - BinaryMetaDataImpl res = new BinaryMetaDataImpl(typeId, newMeta.typeName(), fields, + BinaryMetadata res = new BinaryMetadata(typeId, newMeta.typeName(), fields, newMeta.affinityKeyFieldName()); entry.setValue(res); @@ -861,7 +861,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** {@inheritDoc} */ @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { typeId = in.readInt(); - newMeta = (BinaryMetaDataImpl)in.readObject(); + newMeta = (BinaryMetadata)in.readObject(); } /** {@inheritDoc} */ @@ -873,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(); - final BinaryMetaDataImpl newMeta = evt.getValue(); + final BinaryMetadata newMeta = evt.getValue(); assert newMeta != null : evt; http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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 fdc0e0d..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; @@ -390,7 +390,7 @@ public class PlatformContextImpl implements PlatformContext { else { writer.writeBoolean(true); - Map fields = ((BinaryMetaDataImpl)meta).fieldsMap(); + Map fields = ((BinaryMetadata)meta).fieldsMap(); writer.writeInt(typeId); writer.writeString(meta.typeName()); http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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/b2fd7bfd/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 1ee36d8..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 @@ -47,7 +47,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes * @throws Exception If failed. */ protected static PortableMarshaller createMarshaller() throws Exception { - PortableContext ctx = new PortableContext(new TestMetadataHandler(), new IgniteConfiguration()); + PortableContext ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration()); PortableMarshaller marsh = new PortableMarshaller(); http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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 d10d3f3..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 @@ -45,7 +45,7 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac PortableMarshaller marsh = new PortableMarshaller(); marsh.setContext(new MarshallerContextWithNoStorage()); - PortableContext context = new PortableContext(new TestNoopMetadataHandler(), 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/b2fd7bfd/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 d7007de..a03e73d 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 @@ -2403,7 +2403,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest { protected PortableContext initPortableContext(PortableMarshaller marsh) throws IgniteCheckedException { IgniteConfiguration iCfg = new IgniteConfiguration(); - PortableContext ctx = new PortableContext(new TestNoopMetadataHandler(), iCfg); + PortableContext ctx = new PortableContext(BinaryNoopMetadataHandler.instance(), iCfg); marsh.setContext(new MarshallerContextTestImpl(null)); http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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 ee90beb..1b036d1 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 @@ -36,7 +36,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest { * @return Portable context. */ private PortableContext portableContext() { - return new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration()); + return new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration()); } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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 4fa2324..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 @@ -47,7 +47,7 @@ public abstract class PortableCompactOffsetsAbstractSelfTest extends GridCommonA @Override protected void beforeTest() throws Exception { super.beforeTest(); - ctx = new PortableContext(new TestMetadataHandler(), new IgniteConfiguration()); + ctx = new PortableContext(new TestCachingMetadataHandler(), new IgniteConfiguration()); marsh = new PortableMarshaller(); http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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..cf31c68 --- /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 BinaryMetadataHandler2 { + /** 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/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java deleted file mode 100644 index c25028d..0000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestMetadataHandler.java +++ /dev/null @@ -1,45 +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.portable; - -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryType; - -import java.util.concurrent.ConcurrentHashMap; - -/** - * Test metadata handler. - */ -public class TestMetadataHandler implements PortableMetaDataHandler { - /** 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 " + - TestMetadataHandler.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/b2fd7bfd/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java deleted file mode 100644 index c504365..0000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/portable/TestNoopMetadataHandler.java +++ /dev/null @@ -1,36 +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.portable; - -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryType; - -/** - * No-op metadata handler. - */ -public class TestNoopMetadataHandler implements PortableMetaDataHandler { - /** {@inheritDoc} */ - @Override public void addMeta(int typeId, BinaryType meta) throws BinaryObjectException { - // No-op. - } - - /** {@inheritDoc} */ - @Override public BinaryType metadata(int typeId) throws BinaryObjectException { - return null; - } -} http://git-wip-us.apache.org/repos/asf/ignite/blob/b2fd7bfd/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 c7a80e3..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,8 +19,8 @@ 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.TestNoopMetadataHandler; import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.marshaller.Marshaller; @@ -37,7 +37,7 @@ public class GridPortableCacheEntryMemorySizeSelfTest extends GridCacheEntryMemo marsh.setContext(new MarshallerContextTestImpl(null)); - PortableContext pCtx = new PortableContext(new TestNoopMetadataHandler(), new IgniteConfiguration()); + PortableContext pCtx = new PortableContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration()); IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", pCtx);