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 2534E18742 for ; Fri, 11 Dec 2015 15:55:31 +0000 (UTC) Received: (qmail 49211 invoked by uid 500); 11 Dec 2015 15:55:31 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 49105 invoked by uid 500); 11 Dec 2015 15:55:31 -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 48187 invoked by uid 99); 11 Dec 2015 15:55:30 -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; Fri, 11 Dec 2015 15:55:30 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 4797AE0565; Fri, 11 Dec 2015 15:55:30 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: akuznetsov@apache.org To: commits@ignite.apache.org Date: Fri, 11 Dec 2015 15:55:58 -0000 Message-Id: <21c6a6dea57443fc84bfbba979cba81f@git.apache.org> In-Reply-To: <18bc6ae53e9b47a1baaa96f91bdceb4e@git.apache.org> References: <18bc6ae53e9b47a1baaa96f91bdceb4e@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [30/59] [abbrv] ignite git commit: ignite-2065: rename "portable" packages to "binary" http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java new file mode 100644 index 0000000..cefad9e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java @@ -0,0 +1,180 @@ +/* + * 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.binary; + +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; + + /** Schemas associated with type. */ + private Collection schemas; + + /** Whether this is enum type. */ + private boolean isEnum; + + /** + * 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. + * @param schemas Schemas. + * @param isEnum Enum flag. + */ + public BinaryMetadata(int typeId, String typeName, @Nullable Map fields, + @Nullable String affKeyFieldName, @Nullable Collection schemas, boolean isEnum) { + assert typeName != null; + + this.typeId = typeId; + this.typeName = typeName; + this.fields = fields; + this.affKeyFieldName = affKeyFieldName; + this.schemas = schemas; + this.isEnum = isEnum; + } + + /** + * @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; + } + + /** + * @return Schemas. + */ + public Collection schemas() { + return schemas != null ? schemas : Collections.emptyList(); + } + + /** + * @return {@code True} if this is enum type. + */ + public boolean isEnum() { + return isEnum; + } + + /** + * 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); + U.writeCollection(out, schemas); + out.writeBoolean(isEnum); + } + + /** {@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); + schemas = U.readCollection(in); + isEnum = in.readBoolean(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(BinaryMetadata.class, this); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java new file mode 100644 index 0000000..af99cce --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataCollector.java @@ -0,0 +1,277 @@ +/* + * 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.binary; + +import org.apache.ignite.binary.BinaryIdMapper; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.binary.BinaryWriter; +import org.jetbrains.annotations.Nullable; + +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; + +/** + * Writer for meta data collection. + */ +class BinaryMetadataCollector implements BinaryWriter { + /** Type ID. */ + private final int typeId; + + /** Type name. */ + private final String typeName; + + /** ID mapper. */ + private final BinaryIdMapper idMapper; + + /** Collected metadata. */ + private final Map meta = new HashMap<>(); + + /** Schema builder. */ + private PortableSchema.Builder schemaBuilder = PortableSchema.Builder.newBuilder(); + + /** + * Constructor. + * + * @param typeId Type ID. + * @param typeName Type name. + * @param idMapper ID mapper. + */ + BinaryMetadataCollector(int typeId, String typeName, BinaryIdMapper idMapper) { + this.typeId = typeId; + this.typeName = typeName; + this.idMapper = idMapper; + } + + /** + * @return Field meta data. + */ + Map meta() { + return meta; + } + + /** + * @return Schemas. + */ + PortableSchema schema() { + return schemaBuilder.build(); + } + + /** {@inheritDoc} */ + @Override public void writeByte(String fieldName, byte val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.BYTE); + } + + /** {@inheritDoc} */ + @Override public void writeShort(String fieldName, short val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.SHORT); + } + + /** {@inheritDoc} */ + @Override public void writeInt(String fieldName, int val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.INT); + } + + /** {@inheritDoc} */ + @Override public void writeLong(String fieldName, long val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.LONG); + } + + /** {@inheritDoc} */ + @Override public void writeFloat(String fieldName, float val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.FLOAT); + } + + /** {@inheritDoc} */ + @Override public void writeDouble(String fieldName, double val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.DOUBLE); + } + + /** {@inheritDoc} */ + @Override public void writeChar(String fieldName, char val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.CHAR); + } + + /** {@inheritDoc} */ + @Override public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.BOOLEAN); + } + + /** {@inheritDoc} */ + @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.DECIMAL); + } + + /** {@inheritDoc} */ + @Override public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.STRING); + } + + /** {@inheritDoc} */ + @Override public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.UUID); + } + + /** {@inheritDoc} */ + @Override public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.DATE); + } + + /** {@inheritDoc} */ + @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.TIMESTAMP); + } + + /** {@inheritDoc} */ + @Override public > void writeEnum(String fieldName, T val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.ENUM); + } + + /** {@inheritDoc} */ + @Override public > void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.ENUM_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.OBJECT); + } + + /** {@inheritDoc} */ + @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.BYTE_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.SHORT_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.INT_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.LONG_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.FLOAT_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.DOUBLE_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.CHAR_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.BOOLEAN_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.DECIMAL_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.STRING_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.UUID_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.DATE_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.TIMESTAMP_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.OBJECT_ARR); + } + + /** {@inheritDoc} */ + @Override public void writeCollection(String fieldName, @Nullable Collection col) + throws BinaryObjectException { + add(fieldName, BinaryWriteMode.COL); + } + + /** {@inheritDoc} */ + @Override public void writeMap(String fieldName, @Nullable Map map) throws BinaryObjectException { + add(fieldName, BinaryWriteMode.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, BinaryWriteMode 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) + ']' + ); + } + + schemaBuilder.addField(idMapper.fieldId(typeId, name)); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadataHandler.java new file mode 100644 index 0000000..fea2893 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/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.binary; + +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; +} http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryNoopMetadataHandler.java new file mode 100644 index 0000000..9c0c37d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/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.binary; + +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/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectEx.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectEx.java new file mode 100644 index 0000000..ba15d13 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectEx.java @@ -0,0 +1,30 @@ +/* + * 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.binary; + +import org.apache.ignite.binary.BinaryObject; + +/** + * Extended binary object interface. + */ +public interface BinaryObjectEx extends BinaryObject { + /** + * @return Type ID. + */ + public int typeId(); +} http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java new file mode 100644 index 0000000..d09bc28 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java @@ -0,0 +1,251 @@ +/* + * 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.binary; + +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.IdentityHashMap; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; +import org.apache.ignite.internal.util.typedef.internal.SB; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.binary.BinaryObject; +import org.jetbrains.annotations.Nullable; + +/** + * Internal portable object interface. + */ +public abstract class BinaryObjectExImpl implements BinaryObjectEx { + /** + * @return Length. + */ + public abstract int length(); + + /** + * @return Object start. + */ + public abstract int start(); + + /** + * @return {@code True} if object is array based. + */ + protected abstract boolean hasArray(); + + /** + * @return Object array if object is array based, otherwise {@code null}. + */ + public abstract byte[] array(); + + /** + * @return Object offheap address is object is offheap based, otherwise 0. + */ + public abstract long offheapAddress(); + + /** + * Gets field value. + * + * @param fieldId Field ID. + * @return Field value. + * @throws org.apache.ignite.binary.BinaryObjectException In case of any other error. + */ + @Nullable public abstract F field(int fieldId) throws BinaryObjectException; + + /** {@inheritDoc} */ + @Override public int enumOrdinal() throws BinaryObjectException { + throw new BinaryObjectException("Object is not enum."); + } + + /** + * Get field by offset. + * + * @param fieldOffset Field offset. + * @return Field value. + */ + @Nullable protected abstract F fieldByOrder(int fieldOffset); + + /** + * @param ctx Reader context. + * @param fieldName Field name. + * @return Field value. + */ + @Nullable protected abstract F field(BinaryReaderHandles ctx, String fieldName); + + /** + * Get schema ID. + * + * @return Schema ID. + */ + protected abstract int schemaId(); + + /** + * Create schema for object. + * + * @return Schema. + */ + protected abstract PortableSchema createSchema(); + + /** {@inheritDoc} */ + @Override public BinaryObject clone() throws CloneNotSupportedException { + return (BinaryObject)super.clone(); + } + + /** {@inheritDoc} */ + public boolean equals(Object other) { + if (other == this) + return true; + + if (other == null) + return false; + + if (!(other instanceof BinaryObjectExImpl)) + return false; + + BinaryObjectExImpl otherPo = (BinaryObjectExImpl)other; + + if (length() != otherPo.length() || typeId() != otherPo.typeId()) + return false; + + if (hasArray()) { + if (otherPo.hasArray()) { + int len = length(); + int end = start() + len; + + byte[] arr = array(); + byte[] otherArr = otherPo.array(); + + for (int i = start(), j = otherPo.start(); i < end; i++, j++) { + if (arr[i] != otherArr[j]) + return false; + } + + return true; + } + else { + assert otherPo.offheapAddress() > 0; + + return GridUnsafeMemory.compare(otherPo.offheapAddress() + otherPo.start(), array()); + } + } + else { + assert offheapAddress() > 0; + + if (otherPo.hasArray()) + return GridUnsafeMemory.compare(offheapAddress() + start(), otherPo.array()); + else { + assert otherPo.offheapAddress() > 0; + + return GridUnsafeMemory.compare(offheapAddress() + start(), + otherPo.offheapAddress() + otherPo.start(), + length()); + } + } + } + + /** + * @param ctx Reader context. + * @param handles Handles for already traversed objects. + * @return String representation. + */ + private String toString(BinaryReaderHandles ctx, IdentityHashMap handles) { + int idHash = System.identityHashCode(this); + int hash = hashCode(); + + BinaryType meta; + + try { + meta = type(); + } + catch (BinaryObjectException ignore) { + meta = null; + } + + if (meta == null) + return BinaryObject.class.getSimpleName() + " [idHash=" + idHash + ", hash=" + hash + ", typeId=" + typeId() + ']'; + + handles.put(this, idHash); + + SB buf = new SB(meta.typeName()); + + if (meta.fieldNames() != null) { + buf.a(" [idHash=").a(idHash).a(", hash=").a(hash); + + for (String name : meta.fieldNames()) { + Object val = field(ctx, name); + + buf.a(", ").a(name).a('='); + + if (val instanceof byte[]) + buf.a(Arrays.toString((byte[]) val)); + else if (val instanceof short[]) + buf.a(Arrays.toString((short[])val)); + else if (val instanceof int[]) + buf.a(Arrays.toString((int[])val)); + else if (val instanceof long[]) + buf.a(Arrays.toString((long[])val)); + else if (val instanceof float[]) + buf.a(Arrays.toString((float[])val)); + else if (val instanceof double[]) + buf.a(Arrays.toString((double[])val)); + else if (val instanceof char[]) + buf.a(Arrays.toString((char[])val)); + else if (val instanceof boolean[]) + buf.a(Arrays.toString((boolean[]) val)); + else if (val instanceof BigDecimal[]) + buf.a(Arrays.toString((BigDecimal[])val)); + else { + if (val instanceof BinaryObjectExImpl) { + BinaryObjectExImpl po = (BinaryObjectExImpl)val; + + Integer idHash0 = handles.get(val); + + if (idHash0 != null) { // Circular reference. + BinaryType meta0 = po.type(); + + assert meta0 != null; + + buf.a(meta0.typeName()).a(" [hash=").a(idHash0).a(", ...]"); + } + else + buf.a(po.toString(ctx, handles)); + } + else + buf.a(val); + } + } + + buf.a(']'); + } + + return buf.toString(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + try { + BinaryReaderHandles ctx = new BinaryReaderHandles(); + + ctx.put(start(), this); + + return toString(ctx, new IdentityHashMap()); + } + catch (BinaryObjectException e) { + throw new IgniteException("Failed to create string representation of portable object.", e); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java new file mode 100644 index 0000000..a23330e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java @@ -0,0 +1,570 @@ +/* + * 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.binary; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.util.Date; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.IgniteCodeGeneratingFail; +import org.apache.ignite.internal.binary.streams.PortableHeapInputStream; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.Nullable; + +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Portable object implementation. + */ +@IgniteCodeGeneratingFail // Fields arr and start should not be generated by MessageCodeGenerator. +public final class BinaryObjectImpl extends BinaryObjectExImpl implements Externalizable, KeyCacheObject { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + @GridDirectTransient + private PortableContext ctx; + + /** */ + private byte[] arr; + + /** */ + private int start; + + /** */ + @GridDirectTransient + private Object obj; + + /** */ + @GridDirectTransient + private boolean detachAllowed; + + /** + * For {@link Externalizable}. + */ + public BinaryObjectImpl() { + // No-op. + } + + /** + * @param ctx Context. + * @param arr Array. + * @param start Start. + */ + public BinaryObjectImpl(PortableContext ctx, byte[] arr, int start) { + assert ctx != null; + assert arr != null; + + this.ctx = ctx; + this.arr = arr; + this.start = start; + } + + /** {@inheritDoc} */ + @Override public byte cacheObjectType() { + return TYPE_BINARY; + } + + /** {@inheritDoc} */ + @Override public boolean isPlatformType() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean internal() { + return false; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override public T value(CacheObjectContext ctx, boolean cpy) { + Object obj0 = obj; + + if (obj0 == null || (cpy && needCopy(ctx))) + obj0 = deserializeValue(ctx); + + return (T)obj0; + } + + /** {@inheritDoc} */ + @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException { + if (detached()) + return array(); + + int len = length(); + + byte[] arr0 = new byte[len]; + + U.arrayCopy(arr, start, arr0, 0, len); + + return arr0; + } + + /** {@inheritDoc} */ + @Override public CacheObject prepareForCache(CacheObjectContext ctx) { + if (detached()) + return this; + + return (BinaryObjectImpl)detach(); + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException { + this.ctx = ((CacheObjectBinaryProcessorImpl)ctx.processor()).portableContext(); + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public int length() { + return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.TOTAL_LEN_POS); + } + + /** + * @return Detached portable object. + */ + public BinaryObject detach() { + if (!detachAllowed || detached()) + return this; + + int len = length(); + + byte[] arr0 = new byte[len]; + + U.arrayCopy(arr, start, arr0, 0, len); + + return new BinaryObjectImpl(ctx, arr0, 0); + } + + /** + * @return Detached or not. + */ + public boolean detached() { + return start == 0 && length() == arr.length; + } + + /** + * @param detachAllowed Detach allowed flag. + */ + public void detachAllowed(boolean detachAllowed) { + this.detachAllowed = detachAllowed; + } + + /** + * @return Context. + */ + public PortableContext context() { + return ctx; + } + + /** + * @param ctx Context. + */ + public void context(PortableContext ctx) { + this.ctx = ctx; + } + + /** {@inheritDoc} */ + @Override public byte[] array() { + return arr; + } + + /** {@inheritDoc} */ + @Override public int start() { + return start; + } + + /** {@inheritDoc} */ + @Override public long offheapAddress() { + return 0; + } + + /** {@inheritDoc} */ + @Override protected boolean hasArray() { + return true; + } + + /** {@inheritDoc} */ + @Override public int typeId() { + return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.TYPE_ID_POS); + } + + /** {@inheritDoc} */ + @Nullable @Override public BinaryType type() throws BinaryObjectException { + if (ctx == null) + throw new BinaryObjectException("PortableContext is not set for the object."); + + return ctx.metadata(typeId()); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override public F field(String fieldName) throws BinaryObjectException { + return (F) reader(null).unmarshalField(fieldName); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override public F field(int fieldId) throws BinaryObjectException { + return (F) reader(null).unmarshalField(fieldId); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override protected F fieldByOrder(int order) { + Object val; + + // Calculate field position. + int schemaOffset = PortablePrimitives.readInt(arr, start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS); + + short flags = PortablePrimitives.readShort(arr, start + GridPortableMarshaller.FLAGS_POS); + + int fieldIdLen = PortableUtils.isCompactFooter(flags) ? 0 : PortableUtils.FIELD_ID_LEN; + int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags); + + int fieldOffsetPos = start + schemaOffset + order * (fieldIdLen + fieldOffsetLen) + fieldIdLen; + + int fieldPos; + + if (fieldOffsetLen == PortableUtils.OFFSET_1) + fieldPos = start + ((int)PortablePrimitives.readByte(arr, fieldOffsetPos) & 0xFF); + else if (fieldOffsetLen == PortableUtils.OFFSET_2) + fieldPos = start + ((int)PortablePrimitives.readShort(arr, fieldOffsetPos) & 0xFFFF); + else + fieldPos = start + PortablePrimitives.readInt(arr, fieldOffsetPos); + + // Read header and try performing fast lookup for well-known types (the most common types go first). + byte hdr = PortablePrimitives.readByte(arr, fieldPos); + + switch (hdr) { + case GridPortableMarshaller.INT: + val = PortablePrimitives.readInt(arr, fieldPos + 1); + + break; + + case GridPortableMarshaller.LONG: + val = PortablePrimitives.readLong(arr, fieldPos + 1); + + break; + + case GridPortableMarshaller.BOOLEAN: + val = PortablePrimitives.readBoolean(arr, fieldPos + 1); + + break; + + case GridPortableMarshaller.SHORT: + val = PortablePrimitives.readShort(arr, fieldPos + 1); + + break; + + case GridPortableMarshaller.BYTE: + val = PortablePrimitives.readByte(arr, fieldPos + 1); + + break; + + case GridPortableMarshaller.CHAR: + val = PortablePrimitives.readChar(arr, fieldPos + 1); + + break; + + case GridPortableMarshaller.FLOAT: + val = PortablePrimitives.readFloat(arr, fieldPos + 1); + + break; + + case GridPortableMarshaller.DOUBLE: + val = PortablePrimitives.readDouble(arr, fieldPos + 1); + + break; + + case GridPortableMarshaller.STRING: { + int dataLen = PortablePrimitives.readInt(arr, fieldPos + 1); + + val = new String(arr, fieldPos + 5, dataLen, UTF_8); + + break; + } + + case GridPortableMarshaller.DATE: { + long time = PortablePrimitives.readLong(arr, fieldPos + 1); + + val = new Date(time); + + break; + } + + case GridPortableMarshaller.TIMESTAMP: { + long time = PortablePrimitives.readLong(arr, fieldPos + 1); + int nanos = PortablePrimitives.readInt(arr, fieldPos + 1 + 8); + + Timestamp ts = new Timestamp(time); + + ts.setNanos(ts.getNanos() + nanos); + + val = ts; + + break; + } + + case GridPortableMarshaller.UUID: { + long most = PortablePrimitives.readLong(arr, fieldPos + 1); + long least = PortablePrimitives.readLong(arr, fieldPos + 1 + 8); + + val = new UUID(most, least); + + break; + } + + case GridPortableMarshaller.DECIMAL: { + int scale = PortablePrimitives.readInt(arr, fieldPos + 1); + + int dataLen = PortablePrimitives.readInt(arr, fieldPos + 5); + byte[] data = PortablePrimitives.readByteArray(arr, fieldPos + 9, dataLen); + + BigInteger intVal = new BigInteger(data); + + if (scale < 0) { + scale &= 0x7FFFFFFF; + + intVal = intVal.negate(); + } + + val = new BigDecimal(intVal, scale); + + break; + } + + case GridPortableMarshaller.NULL: + val = null; + + break; + + default: + val = PortableUtils.unmarshal(PortableHeapInputStream.create(arr, fieldPos), ctx, null); + + break; + } + + return (F)val; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override protected F field(BinaryReaderHandles rCtx, String fieldName) { + return (F)reader(rCtx).unmarshalField(fieldName); + } + + /** {@inheritDoc} */ + @Override public boolean hasField(String fieldName) { + return reader(null).findFieldByName(fieldName); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override public T deserialize() throws BinaryObjectException { + Object obj0 = obj; + + if (obj0 == null) + obj0 = deserializeValue(null); + + return (T)obj0; + } + + /** {@inheritDoc} */ + @Override public BinaryObject clone() throws CloneNotSupportedException { + return super.clone(); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.HASH_CODE_POS); + } + + /** {@inheritDoc} */ + @Override protected int schemaId() { + return PortablePrimitives.readInt(arr, start + GridPortableMarshaller.SCHEMA_ID_POS); + } + + /** {@inheritDoc} */ + @Override protected PortableSchema createSchema() { + return reader(null).getOrCreateSchema(); + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeObject(ctx); + + if (detachAllowed) { + int len = length(); + + out.writeInt(len); + out.write(arr, start, len); + out.writeInt(0); + } + else { + out.writeInt(arr.length); + out.write(arr); + out.writeInt(start); + } + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + ctx = (PortableContext)in.readObject(); + + arr = new byte[in.readInt()]; + + in.readFully(arr); + + start = in.readInt(); + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeByteArray("arr", + arr, + detachAllowed ? start : 0, + detachAllowed ? length() : arr.length)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeInt("start", detachAllowed ? 0 : start)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + switch (reader.state()) { + case 0: + arr = reader.readByteArray("arr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + start = reader.readInt("start"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public byte directType() { + return 113; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 3; + } + + /** + * Runs value deserialization regardless of whether obj already has the deserialized value. + * Will set obj if descriptor is configured to keep deserialized values. + * @param coCtx CacheObjectContext. + * @return Object. + */ + private Object deserializeValue(@Nullable CacheObjectContext coCtx) { + // TODO: IGNITE-1272 - Deserialize with proper class loader. + BinaryReaderExImpl reader = reader(null); + + Object obj0 = reader.deserialize(); + + PortableClassDescriptor desc = reader.descriptor(); + + assert desc != null; + + if (coCtx != null && coCtx.storeValue()) + obj = obj0; + + return obj0; + } + + /** + * @param ctx Context. + * @return {@code True} need to copy value returned to user. + */ + private boolean needCopy(CacheObjectContext ctx) { + return ctx.copyOnGet() && obj != null && !ctx.processor().immutable(obj); + } + + /** + * Create new reader for this object. + * + * @param rCtx Reader context. + * @return Reader. + */ + private BinaryReaderExImpl reader(@Nullable BinaryReaderHandles rCtx) { + return new BinaryReaderExImpl(ctx, + PortableHeapInputStream.create(arr, start), + ctx.configuration().getClassLoader(), + rCtx); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java new file mode 100644 index 0000000..27b46c1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java @@ -0,0 +1,430 @@ +/* + * 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.binary; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.internal.binary.streams.PortableOffheapInputStream; +import org.apache.ignite.internal.binary.streams.PortableOffheapInputStream; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; +import org.apache.ignite.internal.util.GridUnsafe; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.Nullable; +import sun.misc.Unsafe; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.util.Date; +import java.util.UUID; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.BOOLEAN; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.BYTE; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.CHAR; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.DATE; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.DECIMAL; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.DOUBLE; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.FLOAT; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.INT; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.LONG; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.NULL; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.SHORT; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.STRING; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.TIMESTAMP; +import static org.apache.ignite.internal.binary.GridPortableMarshaller.UUID; + +/** + * Portable object implementation over offheap memory + */ +public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Externalizable, CacheObject { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private static final Unsafe UNSAFE = GridUnsafe.unsafe(); + + /** */ + private final PortableContext ctx; + + /** */ + private final long ptr; + + /** */ + private final int start; + + /** */ + private final int size; + + /** + * For {@link Externalizable} (not supported). + */ + public BinaryObjectOffheapImpl() { + throw new UnsupportedOperationException(); + } + + /** + * @param ctx Context. + * @param ptr Memory address. + * @param start Object start. + * @param size Memory size. + */ + public BinaryObjectOffheapImpl(PortableContext ctx, long ptr, int start, int size) { + this.ctx = ctx; + this.ptr = ptr; + this.start = start; + this.size = size; + } + + /** + * @return Heap-based copy. + */ + public BinaryObject heapCopy() { + return new BinaryObjectImpl(ctx, U.copyMemory(ptr, size), start); + } + + /** {@inheritDoc} */ + @Override public int typeId() { + return UNSAFE.getInt(ptr + start + GridPortableMarshaller.TYPE_ID_POS); + } + + /** {@inheritDoc} */ + @Override public int length() { + return UNSAFE.getInt(ptr + start + GridPortableMarshaller.TOTAL_LEN_POS); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return UNSAFE.getInt(ptr + start + GridPortableMarshaller.HASH_CODE_POS); + } + + /** {@inheritDoc} */ + @Override protected int schemaId() { + return UNSAFE.getInt(ptr + start + GridPortableMarshaller.SCHEMA_ID_POS); + } + + /** {@inheritDoc} */ + @Override protected PortableSchema createSchema() { + return reader(null).getOrCreateSchema(); + } + + /** {@inheritDoc} */ + @Override public int start() { + return start; + } + + /** {@inheritDoc} */ + @Override public byte[] array() { + return null; + } + + /** {@inheritDoc} */ + @Override public long offheapAddress() { + return ptr; + } + + /** {@inheritDoc} */ + @Override protected boolean hasArray() { + return false; + } + + /** {@inheritDoc} */ + @Nullable @Override public BinaryType type() throws BinaryObjectException { + if (ctx == null) + throw new BinaryObjectException("PortableContext is not set for the object."); + + return ctx.metadata(typeId()); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override public F field(String fieldName) throws BinaryObjectException { + return (F) reader(null).unmarshalField(fieldName); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override public F field(int fieldId) throws BinaryObjectException { + return (F) reader(null).unmarshalField(fieldId); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override protected F fieldByOrder(int order) { + Object val; + + // Calculate field position. + int schemaOffset = PortablePrimitives.readInt(ptr, start + GridPortableMarshaller.SCHEMA_OR_RAW_OFF_POS); + + short flags = PortablePrimitives.readShort(ptr, start + GridPortableMarshaller.FLAGS_POS); + + int fieldIdLen = PortableUtils.isCompactFooter(flags) ? 0 : PortableUtils.FIELD_ID_LEN; + int fieldOffsetLen = PortableUtils.fieldOffsetLength(flags); + + int fieldOffsetPos = start + schemaOffset + order * (fieldIdLen + fieldOffsetLen) + fieldIdLen; + + int fieldPos; + + if (fieldOffsetLen == PortableUtils.OFFSET_1) + fieldPos = start + ((int)PortablePrimitives.readByte(ptr, fieldOffsetPos) & 0xFF); + else if (fieldOffsetLen == PortableUtils.OFFSET_2) + fieldPos = start + ((int)PortablePrimitives.readShort(ptr, fieldOffsetPos) & 0xFFFF); + else + fieldPos = start + PortablePrimitives.readInt(ptr, fieldOffsetPos); + + // Read header and try performing fast lookup for well-known types (the most common types go first). + byte hdr = PortablePrimitives.readByte(ptr, fieldPos); + + switch (hdr) { + case GridPortableMarshaller.INT: + val = PortablePrimitives.readInt(ptr, fieldPos + 1); + + break; + + case GridPortableMarshaller.LONG: + val = PortablePrimitives.readLong(ptr, fieldPos + 1); + + break; + + case GridPortableMarshaller.BOOLEAN: + val = PortablePrimitives.readBoolean(ptr, fieldPos + 1); + + break; + + case GridPortableMarshaller.SHORT: + val = PortablePrimitives.readShort(ptr, fieldPos + 1); + + break; + + case GridPortableMarshaller.BYTE: + val = PortablePrimitives.readByte(ptr, fieldPos + 1); + + break; + + case GridPortableMarshaller.CHAR: + val = PortablePrimitives.readChar(ptr, fieldPos + 1); + + break; + + case GridPortableMarshaller.FLOAT: + val = PortablePrimitives.readFloat(ptr, fieldPos + 1); + + break; + + case GridPortableMarshaller.DOUBLE: + val = PortablePrimitives.readDouble(ptr, fieldPos + 1); + + break; + + case GridPortableMarshaller.STRING: { + int dataLen = PortablePrimitives.readInt(ptr, fieldPos + 1); + byte[] data = PortablePrimitives.readByteArray(ptr, fieldPos + 5, dataLen); + + val = new String(data, UTF_8); + + break; + } + + case GridPortableMarshaller.DATE: { + long time = PortablePrimitives.readLong(ptr, fieldPos + 1); + + val = new Date(time); + + break; + } + + case GridPortableMarshaller.TIMESTAMP: { + long time = PortablePrimitives.readLong(ptr, fieldPos + 1); + int nanos = PortablePrimitives.readInt(ptr, fieldPos + 1 + 8); + + Timestamp ts = new Timestamp(time); + + ts.setNanos(ts.getNanos() + nanos); + + val = ts; + + break; + } + + case GridPortableMarshaller.UUID: { + long most = PortablePrimitives.readLong(ptr, fieldPos + 1); + long least = PortablePrimitives.readLong(ptr, fieldPos + 1 + 8); + + val = new UUID(most, least); + + break; + } + + case GridPortableMarshaller.DECIMAL: { + int scale = PortablePrimitives.readInt(ptr, fieldPos + 1); + + int dataLen = PortablePrimitives.readInt(ptr, fieldPos + 5); + byte[] data = PortablePrimitives.readByteArray(ptr, fieldPos + 9, dataLen); + + BigInteger intVal = new BigInteger(data); + + if (scale < 0) { + scale &= 0x7FFFFFFF; + + intVal = intVal.negate(); + } + + val = new BigDecimal(intVal, scale); + + break; + } + + case GridPortableMarshaller.NULL: + val = null; + + break; + + default: + PortableOffheapInputStream stream = new PortableOffheapInputStream(ptr, size, false); + + stream.position(fieldPos); + + val = PortableUtils.unmarshal(stream, ctx, null); + + break; + } + + return (F)val; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override protected F field(BinaryReaderHandles rCtx, String fieldName) { + return (F)reader(rCtx).unmarshalField(fieldName); + } + + /** {@inheritDoc} */ + @Override public boolean hasField(String fieldName) { + return reader(null).findFieldByName(fieldName); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override public T deserialize() throws BinaryObjectException { + return (T)deserializeValue(); + } + + /** {@inheritDoc} */ + @SuppressWarnings("CloneDoesntCallSuperClone") + @Override public BinaryObject clone() throws CloneNotSupportedException { + return heapCopy(); + } + + /** {@inheritDoc} */ + @Override public byte cacheObjectType() { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean isPlatformType() { + return false; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Nullable @Override public T value(CacheObjectContext ctx, boolean cpy) { + return (T)deserializeValue(); + } + + /** {@inheritDoc} */ + @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public CacheObject prepareForCache(CacheObjectContext ctx) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public byte directType() { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + throw new UnsupportedOperationException(); // To make sure it is not marshalled. + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + throw new UnsupportedOperationException(); // To make sure it is not marshalled. + } + + /** + * @return Deserialized value. + */ + private Object deserializeValue() { + // TODO: IGNITE-1272 - Deserialize with proper class loader. + return reader(null).deserialize(); + } + + /** + * Create new reader for this object. + * + * @param rCtx Reader context. + * @return Reader. + */ + private BinaryReaderExImpl reader(@Nullable BinaryReaderHandles rCtx) { + PortableOffheapInputStream stream = new PortableOffheapInputStream(ptr, size, false); + + stream.position(start); + + return new BinaryReaderExImpl(ctx, stream, null, rCtx); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawReaderEx.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawReaderEx.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawReaderEx.java new file mode 100644 index 0000000..603275e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawReaderEx.java @@ -0,0 +1,33 @@ +/* + * 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.binary; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawReader; +import org.jetbrains.annotations.Nullable; + +/** + * Extended reader interface. + */ +public interface BinaryRawReaderEx extends BinaryRawReader { + /** + * @return Object. + * @throws org.apache.ignite.binary.BinaryObjectException In case of error. + */ + @Nullable public Object readObjectDetached() throws BinaryObjectException; +} http://git-wip-us.apache.org/repos/asf/ignite/blob/1dbf20e0/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawWriterEx.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawWriterEx.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawWriterEx.java new file mode 100644 index 0000000..6e7e0cb --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryRawWriterEx.java @@ -0,0 +1,60 @@ +/* + * 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.binary; + +import org.apache.ignite.internal.binary.streams.PortableOutputStream; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawWriter; +import org.jetbrains.annotations.Nullable; + +/** + * Extended writer interface. + */ +public interface BinaryRawWriterEx extends BinaryRawWriter, AutoCloseable { + /** + * @param obj Object to write. + * @throws org.apache.ignite.binary.BinaryObjectException In case of error. + */ + public void writeObjectDetached(@Nullable Object obj) throws BinaryObjectException; + + /** + * @return Output stream. + */ + public PortableOutputStream out(); + + /** + * Cleans resources. + */ + @Override public void close(); + + /** + * Reserve a room for an integer. + * + * @return Position in the stream where value is to be written. + */ + public int reserveInt(); + + /** + * Write int value at the specific position. + * + * @param pos Position. + * @param val Value. + * @throws org.apache.ignite.binary.BinaryObjectException If failed. + */ + public void writeInt(int pos, int val) throws BinaryObjectException; +}