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 4564818BA6 for ; Sat, 21 Nov 2015 22:45:30 +0000 (UTC) Received: (qmail 80435 invoked by uid 500); 21 Nov 2015 22:45:30 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 80206 invoked by uid 500); 21 Nov 2015 22:45:30 -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 79621 invoked by uid 99); 21 Nov 2015 22:45:29 -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; Sat, 21 Nov 2015 22:45:29 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 5C0E9E0B28; Sat, 21 Nov 2015 22:45:29 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: agoncharuk@apache.org To: commits@ignite.apache.org Date: Sat, 21 Nov 2015 22:45:39 -0000 Message-Id: <097ad3db2cbf40e68615baf17e9a9ffb@git.apache.org> In-Reply-To: <7494ba451bfc4a07a167b7c829ffa1fa@git.apache.org> References: <7494ba451bfc4a07a167b7c829ffa1fa@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [11/55] [abbrv] ignite git commit: IGNITE-1847: Added "field" method to BinaryType and reworked internal metadata handling. IGNITE-1847: Added "field" method to BinaryType and reworked internal metadata handling. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/63d55062 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/63d55062 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/63d55062 Branch: refs/heads/ignite-1.5 Commit: 63d55062790f481f104bd03eaf53c75473233e56 Parents: f3b0be7 Author: vozerov-gridgain Authored: Wed Nov 18 10:24:15 2015 +0300 Committer: vozerov-gridgain Committed: Wed Nov 18 10:24:15 2015 +0300 ---------------------------------------------------------------------- .../org/apache/ignite/binary/BinaryObject.java | 9 - .../org/apache/ignite/binary/BinaryType.java | 11 +- .../internal/portable/BinaryFieldImpl.java | 14 +- .../portable/BinaryMetaDataCollector.java | 99 +++-- .../internal/portable/BinaryMetaDataImpl.java | 150 -------- .../internal/portable/BinaryMetadata.java | 152 ++++++++ .../portable/BinaryMetadataHandler.java | 44 +++ .../portable/BinaryNoopMetadataHandler.java | 53 +++ .../internal/portable/BinaryObjectEx.java | 4 +- .../internal/portable/BinaryObjectImpl.java | 13 - .../portable/BinaryObjectOffheapImpl.java | 13 - .../internal/portable/BinaryTypeImpl.java | 75 ++++ .../portable/PortableClassDescriptor.java | 169 +++++---- .../internal/portable/PortableContext.java | 39 +- .../portable/PortableMetaDataHandler.java | 44 --- .../ignite/internal/portable/PortableUtils.java | 104 +++++- .../builder/BinaryObjectBuilderImpl.java | 32 +- .../portable/builder/PortableValueWithType.java | 9 +- .../CacheObjectBinaryProcessorImpl.java | 360 +++++++------------ .../cache/portable/PortableMetaDataKey.java | 10 +- .../platform/PlatformContextImpl.java | 10 +- .../PlatformDotNetConfigurationClosure.java | 19 +- .../processors/query/GridQueryProcessor.java | 2 +- .../portable/BinaryFieldsAbstractSelfTest.java | 24 +- ...idBinaryObjectBuilderAdditionalSelfTest.java | 2 +- .../GridBinaryObjectBuilderSelfTest.java | 4 +- ...idPortableMarshallerCtxDisabledSelfTest.java | 33 +- .../GridPortableMarshallerSelfTest.java | 13 +- .../portable/GridPortableMetaDataSelfTest.java | 12 +- .../portable/GridPortableWildcardsSelfTest.java | 23 +- .../PortableCompactOffsetsAbstractSelfTest.java | 21 +- .../portable/TestCachingMetadataHandler.java | 45 +++ ...ntNodeBinaryObjectMetadataMultinodeTest.java | 4 +- ...CacheClientNodeBinaryObjectMetadataTest.java | 5 +- ...ridPortableCacheEntryMemorySizeSelfTest.java | 14 +- .../PlatformComputeBinarizableArgTask.java | 2 +- 36 files changed, 840 insertions(+), 797 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java index f48f350..9481618 100644 --- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java +++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java @@ -137,15 +137,6 @@ public interface BinaryObject extends Serializable, Cloneable { public boolean hasField(String fieldName); /** - * Gets field descriptor. - * - * @param fieldName Field name. - * @return Field descriptor. - * @throws BinaryObjectException If failed. - */ - public BinaryField fieldDescriptor(String fieldName) throws BinaryObjectException; - - /** * Gets fully deserialized instance of binary object. * * @return Fully deserialized instance of binary object. http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java index d149fd4..52bb212 100644 --- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java +++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java @@ -38,7 +38,7 @@ public interface BinaryType { * * @return Collection of all field names for this binary type. */ - public Collection fields(); + public Collection fieldNames(); /** * Gets name of the field type for a given field. @@ -49,6 +49,15 @@ public interface BinaryType { public String fieldTypeName(String fieldName); /** + * Get {@link BinaryField} for the given field name. Later this field can be used for fast field's value + * retrieval from concrete {@link BinaryObject}. + * + * @param fieldName Field name. + * @return Binary field. + */ + public BinaryField field(String fieldName); + + /** * Binary objects can optionally specify custom key-affinity mapping in the * configuration. This method returns the name of the field which should be * used for the key-affinity mapping. http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java index b8a25c1..810c820 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.portable; +import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.binary.BinaryObject; @@ -26,6 +27,9 @@ import org.apache.ignite.binary.BinaryField; * Implementation of portable field descriptor. */ public class BinaryFieldImpl implements BinaryField { + /** Type ID. */ + private final int typeId; + /** Well-known object schemas. */ @GridToStringExclude private final PortableSchemaRegistry schemas; @@ -43,11 +47,13 @@ public class BinaryFieldImpl implements BinaryField { * @param fieldName Field name. * @param fieldId Field ID. */ - public BinaryFieldImpl(PortableSchemaRegistry schemas, String fieldName, int fieldId) { + public BinaryFieldImpl(int typeId, PortableSchemaRegistry schemas, String fieldName, int fieldId) { + assert typeId != 0; assert schemas != null; assert fieldName != null; assert fieldId != 0; + this.typeId = typeId; this.schemas = schemas; this.fieldName = fieldName; this.fieldId = fieldId; @@ -82,6 +88,12 @@ public class BinaryFieldImpl implements BinaryField { * @return Field offset. */ private int fieldOrder(BinaryObjectEx obj) { + if (typeId != obj.typeId()) { + throw new BinaryObjectException("Failed to get field because type ID of passed object differs" + + " from type ID this " + BinaryField.class.getSimpleName() + " belongs to [expected=" + typeId + + ", actual=" + obj.typeId() + ']'); + } + int schemaId = obj.schemaId(); PortableSchema schema = schemas.schema(schemaId); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 index b053a55..67e1a0d 100644 --- 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 @@ -35,9 +35,9 @@ import org.jetbrains.annotations.Nullable; /** * Writer for meta data collection. */ -class BinaryMetaDataCollector implements BinaryWriter { +class BinaryMetadataCollector implements BinaryWriter { /** */ - private final Map meta = new HashMap<>(); + private final Map meta = new HashMap<>(); /** */ private final String typeName; @@ -45,176 +45,176 @@ class BinaryMetaDataCollector implements BinaryWriter { /** * @param typeName Type name. */ - BinaryMetaDataCollector(String typeName) { + BinaryMetadataCollector(String typeName) { this.typeName = typeName; } /** * @return Field meta data. */ - Map meta() { + Map meta() { return meta; } /** {@inheritDoc} */ @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException { - add(fieldName, byte.class); + add(fieldName, PortableClassDescriptor.Mode.BYTE); } /** {@inheritDoc} */ @Override public void writeShort(String fieldName, short val) throws BinaryObjectException { - add(fieldName, short.class); + add(fieldName, PortableClassDescriptor.Mode.SHORT); } /** {@inheritDoc} */ @Override public void writeInt(String fieldName, int val) throws BinaryObjectException { - add(fieldName, int.class); + add(fieldName, PortableClassDescriptor.Mode.INT); } /** {@inheritDoc} */ @Override public void writeLong(String fieldName, long val) throws BinaryObjectException { - add(fieldName, long.class); + add(fieldName, PortableClassDescriptor.Mode.LONG); } /** {@inheritDoc} */ @Override public void writeFloat(String fieldName, float val) throws BinaryObjectException { - add(fieldName, float.class); + add(fieldName, PortableClassDescriptor.Mode.FLOAT); } /** {@inheritDoc} */ @Override public void writeDouble(String fieldName, double val) throws BinaryObjectException { - add(fieldName, double.class); + add(fieldName, PortableClassDescriptor.Mode.DOUBLE); } /** {@inheritDoc} */ @Override public void writeChar(String fieldName, char val) throws BinaryObjectException { - add(fieldName, char.class); + add(fieldName, PortableClassDescriptor.Mode.CHAR); } /** {@inheritDoc} */ @Override public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException { - add(fieldName, boolean.class); + add(fieldName, PortableClassDescriptor.Mode.BOOLEAN); } /** {@inheritDoc} */ @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.DECIMAL.typeName()); + add(fieldName, PortableClassDescriptor.Mode.DECIMAL); } /** {@inheritDoc} */ @Override public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException { - add(fieldName, String.class); + add(fieldName, PortableClassDescriptor.Mode.STRING); } /** {@inheritDoc} */ @Override public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException { - add(fieldName, UUID.class); + add(fieldName, PortableClassDescriptor.Mode.UUID); } /** {@inheritDoc} */ @Override public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException { - add(fieldName, Date.class); + add(fieldName, PortableClassDescriptor.Mode.DATE); } /** {@inheritDoc} */ @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException { - add(fieldName, Timestamp.class); + add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP); } /** {@inheritDoc} */ @Override public > void writeEnum(String fieldName, T val) throws BinaryObjectException { - add(fieldName, Enum.class); + add(fieldName, PortableClassDescriptor.Mode.ENUM); } /** {@inheritDoc} */ @Override public > void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException { - add(fieldName, Enum[].class); + add(fieldName, PortableClassDescriptor.Mode.ENUM_ARR); } /** {@inheritDoc} */ @Override public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException { - add(fieldName, Object.class); + add(fieldName, PortableClassDescriptor.Mode.OBJECT); } /** {@inheritDoc} */ @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException { - add(fieldName, byte[].class); + add(fieldName, PortableClassDescriptor.Mode.BYTE_ARR); } /** {@inheritDoc} */ @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException { - add(fieldName, short[].class); + add(fieldName, PortableClassDescriptor.Mode.SHORT_ARR); } /** {@inheritDoc} */ @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException { - add(fieldName, int[].class); + add(fieldName, PortableClassDescriptor.Mode.INT_ARR); } /** {@inheritDoc} */ @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException { - add(fieldName, long[].class); + add(fieldName, PortableClassDescriptor.Mode.LONG_ARR); } /** {@inheritDoc} */ @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException { - add(fieldName, float[].class); + add(fieldName, PortableClassDescriptor.Mode.FLOAT_ARR); } /** {@inheritDoc} */ @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws BinaryObjectException { - add(fieldName, double[].class); + add(fieldName, PortableClassDescriptor.Mode.DOUBLE_ARR); } /** {@inheritDoc} */ @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException { - add(fieldName, char[].class); + add(fieldName, PortableClassDescriptor.Mode.CHAR_ARR); } /** {@inheritDoc} */ @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws BinaryObjectException { - add(fieldName, boolean[].class); + add(fieldName, PortableClassDescriptor.Mode.BOOLEAN_ARR); } /** {@inheritDoc} */ @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws BinaryObjectException { - add(fieldName, PortableClassDescriptor.Mode.DECIMAL_ARR.typeName()); + add(fieldName, PortableClassDescriptor.Mode.DECIMAL_ARR); } /** {@inheritDoc} */ @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws BinaryObjectException { - add(fieldName, String[].class); + add(fieldName, PortableClassDescriptor.Mode.STRING_ARR); } /** {@inheritDoc} */ @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException { - add(fieldName, UUID[].class); + add(fieldName, PortableClassDescriptor.Mode.UUID_ARR); } /** {@inheritDoc} */ @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException { - add(fieldName, Date[].class); + add(fieldName, PortableClassDescriptor.Mode.DATE_ARR); } /** {@inheritDoc} */ @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException { - add(fieldName, Timestamp[].class); + add(fieldName, PortableClassDescriptor.Mode.TIMESTAMP_ARR); } /** {@inheritDoc} */ @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException { - add(fieldName, Object[].class); + add(fieldName, PortableClassDescriptor.Mode.OBJECT_ARR); } /** {@inheritDoc} */ @Override public void writeCollection(String fieldName, @Nullable Collection col) throws BinaryObjectException { - add(fieldName, Collection.class); + add(fieldName, PortableClassDescriptor.Mode.COL); } /** {@inheritDoc} */ @Override public void writeMap(String fieldName, @Nullable Map map) throws BinaryObjectException { - add(fieldName, Map.class); + add(fieldName, PortableClassDescriptor.Mode.MAP); } /** {@inheritDoc} */ @@ -230,32 +230,23 @@ class BinaryMetaDataCollector implements BinaryWriter { /** * @param name Field name. - * @param fieldType Field type. - * @throws org.apache.ignite.binary.BinaryObjectException In case of error. + * @param mode Field mode. + * @throws BinaryObjectException In case of error. */ - private void add(String name, Class fieldType) throws BinaryObjectException { - assert fieldType != null; - - add(name, fieldType.getSimpleName()); - } - - /** - * @param name Field name. - * @param fieldTypeName Field type name. - * @throws org.apache.ignite.binary.BinaryObjectException In case of error. - */ - private void add(String name, String fieldTypeName) throws BinaryObjectException { + private void add(String name, PortableClassDescriptor.Mode mode) throws BinaryObjectException { assert name != null; - String oldFieldTypeName = meta.put(name, fieldTypeName); + int fieldTypeId = mode.typeId(); + + Integer oldFieldTypeId = meta.put(name, fieldTypeId); - if (oldFieldTypeName != null && !oldFieldTypeName.equals(fieldTypeName)) { + if (oldFieldTypeId != null && !oldFieldTypeId.equals(fieldTypeId)) { throw new BinaryObjectException( "Field is written twice with different types [" + "typeName=" + typeName + ", fieldName=" + name + - ", fieldTypeName1=" + oldFieldTypeName + - ", fieldTypeName2=" + fieldTypeName + + ", fieldTypeName1=" + PortableUtils.fieldTypeName(oldFieldTypeId) + + ", fieldTypeName2=" + PortableUtils.fieldTypeName(fieldTypeId) + ']' ); } http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 18f538b..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetaDataImpl.java +++ /dev/null @@ -1,150 +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.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.Binarylizable; -import org.apache.ignite.binary.BinaryType; -import org.apache.ignite.binary.BinaryRawReader; -import org.apache.ignite.binary.BinaryRawWriter; -import org.apache.ignite.binary.BinaryReader; -import org.apache.ignite.binary.BinaryWriter; -import org.jetbrains.annotations.Nullable; - -/** - * Portable meta data implementation. - */ -public class BinaryMetaDataImpl implements BinaryType, Binarylizable, Externalizable { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - private String typeName; - - /** */ - @GridToStringInclude - private Map fields; - - /** */ - private volatile Map fldIdToName; - - /** */ - private String affKeyFieldName; - - /** - * For {@link Externalizable}. - */ - public BinaryMetaDataImpl() { - // No-op. - } - - /** - * @param typeName Type name. - * @param fields Fields map. - * @param affKeyFieldName Affinity key field name. - */ - public BinaryMetaDataImpl(String typeName, @Nullable Map fields, - @Nullable String affKeyFieldName) { - assert typeName != null; - - this.typeName = typeName; - this.fields = fields; - this.affKeyFieldName = affKeyFieldName; - } - - /** {@inheritDoc} */ - @Override public String typeName() { - return typeName; - } - - /** {@inheritDoc} */ - @Override public Collection fields() { - return fields != null ? fields.keySet() : Collections.emptyList(); - } - - /** - * @return Fields. - */ - public Map fields0() { - return fields != null ? fields : Collections.emptyMap(); - } - - /** {@inheritDoc} */ - @Nullable @Override public String fieldTypeName(String fieldName) { - return fields != null ? fields.get(fieldName) : null; - } - - /** {@inheritDoc} */ - @Nullable @Override public String affinityKeyFieldName() { - return affKeyFieldName; - } - - /** - * @return Fields meta data. - */ - public Map fieldsMeta() { - return fields != null ? fields : Collections.emptyMap(); - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - U.writeString(out, typeName); - U.writeMap(out, fields); - U.writeString(out, affKeyFieldName); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - typeName = U.readString(in); - fields = U.readMap(in); - affKeyFieldName = U.readString(in); - } - - /** {@inheritDoc} */ - @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { - BinaryRawWriter raw = writer.rawWriter(); - - raw.writeString(typeName); - raw.writeString(affKeyFieldName); - raw.writeMap(fields); - } - - /** {@inheritDoc} */ - @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { - BinaryRawReader raw = reader.rawReader(); - - typeName = raw.readString(); - affKeyFieldName = raw.readString(); - fields = raw.readMap(); - } - - /** {@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/63d55062/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/63d55062/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler.java new file mode 100644 index 0000000..add8c2d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMetadataHandler.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 BinaryMetadataHandler { + /** + * 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/63d55062/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..c4fc5e3 --- /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 BinaryMetadataHandler { + /** 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/63d55062/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java index 50b9d42..b3512ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java @@ -176,10 +176,10 @@ public abstract class BinaryObjectEx implements BinaryObject { SB buf = new SB(meta.typeName()); - if (meta.fields() != null) { + if (meta.fieldNames() != null) { buf.a(" [hash=").a(idHash); - for (String name : meta.fields()) { + for (String name : meta.fieldNames()) { Object val = field(ctx, name); buf.a(", ").a(name).a('='); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java index 800ca40..d432ea0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java @@ -462,19 +462,6 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz } /** {@inheritDoc} */ - @Override public BinaryField fieldDescriptor(String fieldName) throws BinaryObjectException { - A.notNull(fieldName, "fieldName"); - - int typeId = typeId(); - - PortableSchemaRegistry schemaReg = ctx.schemaRegistry(typeId); - - int fieldId = ctx.userTypeIdMapper(typeId).fieldId(typeId, fieldName); - - return new BinaryFieldImpl(schemaReg, fieldName, fieldId); - } - - /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { out.writeObject(ctx); http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java index 9b6735f..f7cb844 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java @@ -140,19 +140,6 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali } /** {@inheritDoc} */ - @Override public BinaryField fieldDescriptor(String fieldName) throws BinaryObjectException { - A.notNull(fieldName, "fieldName"); - - int typeId = typeId(); - - PortableSchemaRegistry schemaReg = ctx.schemaRegistry(typeId); - - int fieldId = ctx.userTypeIdMapper(typeId).fieldId(typeId, fieldName); - - return new BinaryFieldImpl(schemaReg, fieldName, fieldId); - } - - /** {@inheritDoc} */ @Override public int start() { return start; } http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 new file mode 100644 index 0000000..40b6252 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java @@ -0,0 +1,75 @@ +/* + * 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.BinaryType; + +import java.util.Collection; + +/** + * Binary type implementation. + */ +public class BinaryTypeImpl implements BinaryType { + /** Portable context. */ + private final PortableContext ctx; + + /** Type metadata. */ + private final BinaryMetadata meta; + + /** + * Constructor. + * + * @param ctx Portable context. + * @param meta Type metadata. + */ + public BinaryTypeImpl(PortableContext ctx, BinaryMetadata meta) { + this.ctx = ctx; + this.meta = meta; + } + + /** {@inheritDoc} */ + @Override public String typeName() { + return meta.typeName(); + } + + /** {@inheritDoc} */ + @Override public Collection fieldNames() { + return meta.fields(); + } + + /** {@inheritDoc} */ + @Override public String fieldTypeName(String fieldName) { + return meta.fieldTypeName(fieldName); + } + + /** {@inheritDoc} */ + @Override public BinaryFieldImpl field(String fieldName) { + return ctx.createField(meta.typeId(), fieldName); + } + + public String affinityKeyFieldName() { + return meta.affinityKeyFieldName(); + } + + /** + * @return Metadata. + */ + public BinaryMetadata metadata() { + return meta; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 c9870b4..225e0ba 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 @@ -92,7 +92,7 @@ public class PortableClassDescriptor { private final Method readResolveMtd; /** */ - private final Map fieldsMeta; + private final Map fieldsMeta; /** */ private final boolean keepDeserialized; @@ -182,7 +182,7 @@ public class PortableClassDescriptor { case UUID_ARR: case DATE_ARR: case TIMESTAMP_ARR: - case OBJ_ARR: + case OBJECT_ARR: case COL: case MAP: case MAP_ENTRY: @@ -210,7 +210,7 @@ public class PortableClassDescriptor { ctor = constructor(cls); fields = new ArrayList<>(); - fieldsMeta = metaDataEnabled ? new HashMap() : null; + fieldsMeta = metaDataEnabled ? new HashMap() : null; Collection names = new HashSet<>(); Collection ids = new HashSet<>(); @@ -237,7 +237,7 @@ public class PortableClassDescriptor { fields.add(fieldInfo); if (metaDataEnabled) - fieldsMeta.put(name, fieldInfo.fieldMode().typeName()); + fieldsMeta.put(name, fieldInfo.fieldMode().typeId()); } } } @@ -283,7 +283,7 @@ public class PortableClassDescriptor { /** * @return Fields meta data. */ - Map fieldsMeta() { + Map fieldsMeta() { return fieldsMeta; } @@ -337,6 +337,7 @@ public class PortableClassDescriptor { /** * @return portableReadResolve() method */ + @SuppressWarnings("UnusedDeclaration") @Nullable Method getReadResolveMethod() { return readResolveMtd; } @@ -430,66 +431,66 @@ public class PortableClassDescriptor { break; case SHORT_ARR: - writer.doWriteShortArray((short[])obj); + writer.doWriteShortArray((short[]) obj); break; case INT_ARR: - writer.doWriteIntArray((int[])obj); + writer.doWriteIntArray((int[]) obj); break; case LONG_ARR: - writer.doWriteLongArray((long[])obj); + writer.doWriteLongArray((long[]) obj); break; case FLOAT_ARR: - writer.doWriteFloatArray((float[])obj); + writer.doWriteFloatArray((float[]) obj); break; case DOUBLE_ARR: - writer.doWriteDoubleArray((double[])obj); + writer.doWriteDoubleArray((double[]) obj); break; case CHAR_ARR: - writer.doWriteCharArray((char[])obj); + writer.doWriteCharArray((char[]) obj); break; case BOOLEAN_ARR: - writer.doWriteBooleanArray((boolean[])obj); + writer.doWriteBooleanArray((boolean[]) obj); break; case DECIMAL_ARR: - writer.doWriteDecimalArray((BigDecimal[])obj); + writer.doWriteDecimalArray((BigDecimal[]) obj); break; case STRING_ARR: - writer.doWriteStringArray((String[])obj); + writer.doWriteStringArray((String[]) obj); break; case UUID_ARR: - writer.doWriteUuidArray((UUID[])obj); + writer.doWriteUuidArray((UUID[]) obj); break; case DATE_ARR: - writer.doWriteDateArray((Date[])obj); + writer.doWriteDateArray((Date[]) obj); break; case TIMESTAMP_ARR: - writer.doWriteTimestampArray((Timestamp[])obj); + writer.doWriteTimestampArray((Timestamp[]) obj); break; - case OBJ_ARR: + case OBJECT_ARR: writer.doWriteObjectArray((Object[])obj); break; @@ -543,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); + BinaryMetadataCollector metaCollector = new BinaryMetadataCollector(typeName); if (serializer != null) serializer.writeBinary(obj, metaCollector); @@ -711,6 +712,7 @@ public class PortableClassDescriptor { * @return Constructor. * @throws org.apache.ignite.binary.BinaryObjectException If constructor doesn't exist. */ + @SuppressWarnings("ConstantConditions") @Nullable private static Constructor constructor(Class cls) throws BinaryObjectException { assert cls != null; @@ -739,11 +741,8 @@ public class PortableClassDescriptor { Method writeObj = cls.getDeclaredMethod("writeObject", ObjectOutputStream.class); Method readObj = cls.getDeclaredMethod("readObject", ObjectInputStream.class); - if (!Modifier.isStatic(writeObj.getModifiers()) && !Modifier.isStatic(readObj.getModifiers()) && - writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class) - use = true; - else - use = false; + use = !Modifier.isStatic(writeObj.getModifiers()) && !Modifier.isStatic(readObj.getModifiers()) && + writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class; } catch (NoSuchMethodException e) { use = false; @@ -813,7 +812,7 @@ public class PortableClassDescriptor { else if (cls == Timestamp[].class) return Mode.TIMESTAMP_ARR; else if (cls.isArray()) - return cls.getComponentType().isEnum() ? Mode.ENUM_ARR : Mode.OBJ_ARR; + return cls.getComponentType().isEnum() ? Mode.ENUM_ARR : Mode.OBJECT_ARR; else if (cls == BinaryObjectImpl.class) return Mode.PORTABLE_OBJ; else if (Binarylizable.class.isAssignableFrom(cls)) @@ -961,66 +960,66 @@ public class PortableClassDescriptor { break; case SHORT_ARR: - writer.writeShortArrayField((short[])val); + writer.writeShortArrayField((short[]) val); break; case INT_ARR: - writer.writeIntArrayField((int[])val); + writer.writeIntArrayField((int[]) val); break; case LONG_ARR: - writer.writeLongArrayField((long[])val); + writer.writeLongArrayField((long[]) val); break; case FLOAT_ARR: - writer.writeFloatArrayField((float[])val); + writer.writeFloatArrayField((float[]) val); break; case DOUBLE_ARR: - writer.writeDoubleArrayField((double[])val); + writer.writeDoubleArrayField((double[]) val); break; case CHAR_ARR: - writer.writeCharArrayField((char[])val); + writer.writeCharArrayField((char[]) val); break; case BOOLEAN_ARR: - writer.writeBooleanArrayField((boolean[])val); + writer.writeBooleanArrayField((boolean[]) val); break; case DECIMAL_ARR: - writer.writeDecimalArrayField((BigDecimal[])val); + writer.writeDecimalArrayField((BigDecimal[]) val); break; case STRING_ARR: - writer.writeStringArrayField((String[])val); + writer.writeStringArrayField((String[]) val); break; case UUID_ARR: - writer.writeUuidArrayField((UUID[])val); + writer.writeUuidArrayField((UUID[]) val); break; case DATE_ARR: - writer.writeDateArrayField((Date[])val); + writer.writeDateArrayField((Date[]) val); break; case TIMESTAMP_ARR: - writer.writeTimestampArrayField((Timestamp[])val); + writer.writeTimestampArrayField((Timestamp[]) val); break; - case OBJ_ARR: + case OBJECT_ARR: writer.writeObjectArrayField((Object[])val); break; @@ -1211,7 +1210,7 @@ public class PortableClassDescriptor { break; - case OBJ_ARR: + case OBJECT_ARR: val = reader.readObjectArray(id); break; @@ -1275,134 +1274,134 @@ public class PortableClassDescriptor { /** */ enum Mode { /** */ - BYTE("byte"), + BYTE(GridPortableMarshaller.BYTE), /** */ - SHORT("short"), + SHORT(GridPortableMarshaller.SHORT), /** */ - INT("int"), + INT(GridPortableMarshaller.INT), /** */ - LONG("long"), + LONG(GridPortableMarshaller.LONG), /** */ - FLOAT("float"), + FLOAT(GridPortableMarshaller.FLOAT), /** */ - DOUBLE("double"), + DOUBLE(GridPortableMarshaller.DOUBLE), /** */ - CHAR("char"), + CHAR(GridPortableMarshaller.CHAR), /** */ - BOOLEAN("boolean"), + BOOLEAN(GridPortableMarshaller.BOOLEAN), /** */ - DECIMAL("decimal"), + DECIMAL(GridPortableMarshaller.DECIMAL), /** */ - STRING("String"), + STRING(GridPortableMarshaller.STRING), /** */ - UUID("UUID"), + UUID(GridPortableMarshaller.UUID), /** */ - DATE("Date"), + DATE(GridPortableMarshaller.DATE), /** */ - TIMESTAMP("Timestamp"), + TIMESTAMP(GridPortableMarshaller.TIMESTAMP), /** */ - BYTE_ARR("byte[]"), + BYTE_ARR(GridPortableMarshaller.BYTE_ARR), /** */ - SHORT_ARR("short[]"), + SHORT_ARR(GridPortableMarshaller.SHORT_ARR), /** */ - INT_ARR("int[]"), + INT_ARR(GridPortableMarshaller.INT_ARR), /** */ - LONG_ARR("long[]"), + LONG_ARR(GridPortableMarshaller.LONG_ARR), /** */ - FLOAT_ARR("float[]"), + FLOAT_ARR(GridPortableMarshaller.FLOAT_ARR), /** */ - DOUBLE_ARR("double[]"), + DOUBLE_ARR(GridPortableMarshaller.DOUBLE_ARR), /** */ - CHAR_ARR("char[]"), + CHAR_ARR(GridPortableMarshaller.CHAR_ARR), /** */ - BOOLEAN_ARR("boolean[]"), + BOOLEAN_ARR(GridPortableMarshaller.BOOLEAN_ARR), /** */ - DECIMAL_ARR("decimal[]"), + DECIMAL_ARR(GridPortableMarshaller.DECIMAL_ARR), /** */ - STRING_ARR("String[]"), + STRING_ARR(GridPortableMarshaller.STRING_ARR), /** */ - UUID_ARR("UUID[]"), + UUID_ARR(GridPortableMarshaller.UUID_ARR), /** */ - DATE_ARR("Date[]"), + DATE_ARR(GridPortableMarshaller.DATE_ARR), /** */ - TIMESTAMP_ARR("Timestamp[]"), + TIMESTAMP_ARR(GridPortableMarshaller.TIMESTAMP_ARR), /** */ - OBJ_ARR("Object[]"), + OBJECT_ARR(GridPortableMarshaller.OBJ_ARR), /** */ - COL("Collection"), + COL(GridPortableMarshaller.COL), /** */ - MAP("Map"), + MAP(GridPortableMarshaller.MAP), /** */ - MAP_ENTRY("Entry"), + MAP_ENTRY(GridPortableMarshaller.MAP_ENTRY), /** */ - PORTABLE_OBJ("Object"), + PORTABLE_OBJ(GridPortableMarshaller.OBJ), /** */ - ENUM("Enum"), + ENUM(GridPortableMarshaller.ENUM), /** */ - ENUM_ARR("Enum[]"), + ENUM_ARR(GridPortableMarshaller.ENUM_ARR), /** */ - CLASS("Class"), + CLASS(GridPortableMarshaller.CLASS), /** */ - PORTABLE("Object"), + PORTABLE(GridPortableMarshaller.PORTABLE_OBJ), /** */ - EXTERNALIZABLE("Object"), + EXTERNALIZABLE(GridPortableMarshaller.OBJ), /** */ - OBJECT("Object"), + OBJECT(GridPortableMarshaller.OBJ), /** */ - EXCLUSION("Exclusion"); + EXCLUSION(GridPortableMarshaller.OBJ); /** */ - private final String typeName; + private final int typeId; /** - * @param typeName Type name. + * @param typeId Type ID. */ - Mode(String typeName) { - this.typeName = typeName; + Mode(int typeId) { + this.typeId = typeId; } /** - * @return Type name. + * @return Type ID. */ - String typeName() { - return typeName; + int typeId() { + return typeId; } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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 928be63..86578ad 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 BinaryMetadataHandler metaHnd; /** */ private MarshallerContext marshCtx; @@ -165,8 +165,9 @@ public class PortableContext implements Externalizable { * @param metaHnd Meta data handler. * @param igniteCfg Ignite configuration. */ - public PortableContext(PortableMetaDataHandler metaHnd, @Nullable IgniteConfiguration igniteCfg) { + public PortableContext(BinaryMetadataHandler metaHnd, IgniteConfiguration igniteCfg) { assert metaHnd != null; + assert igniteCfg != null; this.metaHnd = metaHnd; this.igniteCfg = igniteCfg; @@ -202,7 +203,6 @@ public class PortableContext implements Externalizable { registerPredefinedType(Date.class, GridPortableMarshaller.DATE); registerPredefinedType(Timestamp.class, GridPortableMarshaller.TIMESTAMP); registerPredefinedType(UUID.class, GridPortableMarshaller.UUID); - // TODO: How to handle timestamp? It has the same ID in .Net. registerPredefinedType(byte[].class, GridPortableMarshaller.BYTE_ARR); registerPredefinedType(short[].class, GridPortableMarshaller.SHORT_ARR); @@ -237,9 +237,6 @@ public class PortableContext implements Externalizable { registerPredefinedType(T2.class, 62); // IDs range [200..1000] is used by Ignite internal APIs. - - registerPredefinedType(BinaryObjectImpl.class, 200); - registerPredefinedType(BinaryMetaDataImpl.class, 201); } /** @@ -570,7 +567,7 @@ public class PortableContext implements Externalizable { mappers.putIfAbsent(typeId, idMapper); - metaHnd.addMeta(typeId, new BinaryMetaDataImpl(typeName, desc.fieldsMeta(), null)); + metaHnd.addMeta(typeId, new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), null).wrap(this)); return desc; } @@ -752,7 +749,7 @@ public class PortableContext implements Externalizable { typeMappers.put(typeName, idMapper); - Map fieldsMeta = null; + Map fieldsMeta = null; if (cls != null) { PortableClassDescriptor desc = new PortableClassDescriptor( @@ -777,7 +774,22 @@ public class PortableContext implements Externalizable { descByCls.put(cls, desc); } - metaHnd.addMeta(id, new BinaryMetaDataImpl(typeName, fieldsMeta, affKeyFieldName)); + metaHnd.addMeta(id, new BinaryMetadata(id, typeName, fieldsMeta, affKeyFieldName).wrap(this)); + } + + /** + * Create binary field. + * + * @param typeId Type ID. + * @param fieldName Field name. + * @return Binary field. + */ + public BinaryFieldImpl createField(int typeId, String fieldName) { + PortableSchemaRegistry schemaReg = schemaRegistry(typeId); + + int fieldId = userTypeIdMapper(typeId).fieldId(typeId, fieldName); + + return new BinaryFieldImpl(typeId, schemaReg, fieldName, fieldId); } /** @@ -816,8 +828,8 @@ public class PortableContext implements Externalizable { * @param fields Fields map. * @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(typeName, fields, null)); + public void updateMetaData(int typeId, String typeName, Map fields) throws BinaryObjectException { + updateMetaData(typeId, new BinaryMetadata(typeId, typeName, fields, null)); } /** @@ -825,8 +837,8 @@ 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 { - metaHnd.addMeta(typeId, meta); + public void updateMetaData(int typeId, BinaryMetadata meta) throws BinaryObjectException { + metaHnd.addMeta(typeId, meta.wrap(this)); } /** @@ -890,6 +902,7 @@ public class PortableContext implements Externalizable { * @param clsName Class name. * @return Type name. */ + @SuppressWarnings("ResultOfMethodCallIgnored") public static String typeName(String clsName) { assert clsName != null; http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/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/63d55062/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java index fe97e7e..31f2bf9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java @@ -49,6 +49,7 @@ 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; @@ -69,6 +70,7 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENT import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL; 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.PROTO_VER; import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT; import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT_ARR; @@ -116,6 +118,92 @@ public class PortableUtils { /** Offset which fits into 4 bytes. */ public static final int OFFSET_4 = 4; + /** Field type names. */ + private static final String[] FIELD_TYPE_NAMES; + + 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; + } + /** * Write flags. * @@ -471,22 +559,6 @@ public class PortableUtils { } /** - * Tells whether provided type is portable or a collection. - * - * @param cls Class to check. - * @return Whether type is portable or a collection. - */ - public static boolean isPortableOrCollectionType(Class cls) { - assert cls != null; - - return isPortableType(cls) || - cls == Object[].class || - Collection.class.isAssignableFrom(cls) || - Map.class.isAssignableFrom(cls) || - Map.Entry.class.isAssignableFrom(cls); - } - - /** * Tells whether provided type is portable. * * @param cls Class to check. http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java index 777d30b..ca8f09b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java @@ -17,15 +17,10 @@ package org.apache.ignite.internal.portable.builder; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.Set; +import org.apache.ignite.binary.BinaryInvalidTypeException; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryInvalidTypeException; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.internal.portable.BinaryObjectImpl; import org.apache.ignite.internal.portable.BinaryObjectOffheapImpl; @@ -33,13 +28,18 @@ import org.apache.ignite.internal.portable.BinaryWriterExImpl; import org.apache.ignite.internal.portable.GridPortableMarshaller; import org.apache.ignite.internal.portable.PortableContext; import org.apache.ignite.internal.portable.PortableUtils; -import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.util.GridArgumentCheck; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; + import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN; import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLAGS_POS; import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_CODE_POS; @@ -284,7 +284,7 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder { if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) { BinaryType metadata = ctx.metaData(typeId); - Map newFldsMetadata = null; + Map newFldsMetadata = null; for (Map.Entry entry : assignedVals.entrySet()) { Object val = entry.getValue(); @@ -305,15 +305,14 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder { String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name); - String newFldTypeName; + int newFldTypeId; if (val instanceof PortableValueWithType) - newFldTypeName = ((PortableValueWithType) val).typeName(); - else { - byte type = PortableUtils.typeByClass(val.getClass()); + newFldTypeId = ((PortableValueWithType) val).typeId(); + else + newFldTypeId = PortableUtils.typeByClass(val.getClass()); - newFldTypeName = CacheObjectBinaryProcessorImpl.fieldTypeName(type); - } + String newFldTypeName = PortableUtils.fieldTypeName(newFldTypeId); if (oldFldTypeName == null) { // It's a new field, we have to add it to metadata. @@ -321,11 +320,10 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder { if (newFldsMetadata == null) newFldsMetadata = new HashMap<>(); - newFldsMetadata.put(name, newFldTypeName); + newFldsMetadata.put(name, PortableUtils.fieldTypeId(newFldTypeName)); } else { - String objTypeName = - CacheObjectBinaryProcessorImpl.FIELD_TYPE_NAMES[GridPortableMarshaller.OBJ]; + String objTypeName = PortableUtils.fieldTypeName(GridPortableMarshaller.OBJ); if (!objTypeName.equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) { throw new BinaryObjectException( http://git-wip-us.apache.org/repos/asf/ignite/blob/63d55062/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java index 5ea8e62..5d66328 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.portable.builder; import org.apache.ignite.internal.portable.BinaryWriterExImpl; -import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -48,9 +47,11 @@ class PortableValueWithType implements PortableLazyValue { ctx.writeValue(writer, val); } - /** {@inheritDoc} */ - public String typeName() { - return CacheObjectBinaryProcessorImpl.fieldTypeName(type); + /** + * @return Type ID. + */ + public int typeId() { + return type; } /** {@inheritDoc} */