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 5704918A62 for ; Tue, 25 Aug 2015 09:08:32 +0000 (UTC) Received: (qmail 54128 invoked by uid 500); 25 Aug 2015 09:08:32 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 54087 invoked by uid 500); 25 Aug 2015 09:08:32 -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 54077 invoked by uid 99); 25 Aug 2015 09:08:32 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 25 Aug 2015 09:08:32 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id B1EE6DFB8F; Tue, 25 Aug 2015 09:08:31 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Tue, 25 Aug 2015 09:08:39 -0000 Message-Id: <0133e3317abc46d2af5b76c1a278df70@git.apache.org> In-Reply-To: <01ae7429b82644bd8e8f218a32dde56a@git.apache.org> References: <01ae7429b82644bd8e8f218a32dde56a@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [09/19] ignite git commit: ignite-1258: portable objects API support in Ignite http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java new file mode 100644 index 0000000..d819a56 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.portable; + +import org.apache.ignite.*; +import org.apache.ignite.internal.*; +import org.apache.ignite.internal.processors.cacheobject.*; +import org.apache.ignite.portable.*; + +import org.jetbrains.annotations.*; + +import java.util.*; + +/** + * {@link IgnitePortables} implementation. + */ +public class IgnitePortablesImpl implements IgnitePortables { + /** */ + private GridKernalContext ctx; + + /** */ + private CacheObjectPortableProcessor proc; + + /** + * @param ctx Context. + */ + public IgnitePortablesImpl(GridKernalContext ctx, CacheObjectPortableProcessor proc) { + this.ctx = ctx; + + this.proc = proc; + } + + /** {@inheritDoc} */ + @Override public int typeId(String typeName) { + guard(); + + try { + return proc.typeId(typeName); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public T toPortable(@Nullable Object obj) throws PortableException { + guard(); + + try { + return (T)proc.marshalToPortable(obj); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public PortableBuilder builder(int typeId) { + guard(); + + try { + return proc.builder(typeId); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public PortableBuilder builder(String typeName) { + guard(); + + try { + return proc.builder(typeName); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public PortableBuilder builder(PortableObject portableObj) { + guard(); + + try { + return proc.builder(portableObj); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Nullable @Override public PortableMetadata metadata(Class cls) throws PortableException { + guard(); + + try { + return proc.metadata(proc.typeId(cls.getName())); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Nullable @Override public PortableMetadata metadata(String typeName) throws PortableException { + guard(); + + try { + return proc.metadata(proc.typeId(typeName)); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Nullable @Override public PortableMetadata metadata(int typeId) throws PortableException { + guard(); + + try { + return proc.metadata(typeId); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public Collection metadata() throws PortableException { + guard(); + + try { + return proc.metadata(); + } + finally { + unguard(); + } + } + + /** + * @return Portable processor. + */ + public IgniteCacheObjectProcessor processor() { + return proc; + } + + /** + * ctx.gateway().readLock() + */ + private void guard() { + ctx.gateway().readLock(); + } + + /** + * ctx.gateway().readUnlock() + */ + private void unguard() { + ctx.gateway().readUnlock(); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/PortableMetaDataKey.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/PortableMetaDataKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/PortableMetaDataKey.java new file mode 100644 index 0000000..d2d53cd --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/PortableMetaDataKey.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.portable; + +import org.apache.ignite.internal.processors.cache.*; +import org.apache.ignite.internal.util.typedef.internal.*; + +import java.io.*; + +/** + * Key for portable meta data. + */ +class PortableMetaDataKey extends GridCacheUtilityKey implements Externalizable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private int typeId; + + /** + * For {@link Externalizable}. + */ + public PortableMetaDataKey() { + // No-op. + } + + /** + * @param typeId Type ID. + */ + PortableMetaDataKey(int typeId) { + this.typeId = typeId; + } + + /** + * @return Type id. + */ + public int typeId() { + return typeId; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeInt(typeId); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + typeId = in.readInt(); + } + + /** {@inheritDoc} */ + @Override protected boolean equalsx(PortableMetaDataKey key) { + return typeId == key.typeId; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return typeId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PortableMetaDataKey.class, this); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/package-info.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/package-info.java new file mode 100644 index 0000000..6c30811 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Implementation of portable processor. + */ +package org.apache.ignite.internal.processors.cache.portable; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java index 02fe679..f11df41 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java @@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.store; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.*; +import org.apache.ignite.marshaller.portable.*; /** * Default store manager implementation. @@ -53,6 +54,6 @@ public class CacheOsStoreManager extends GridCacheStoreManagerAdapter { /** {@inheritDoc} */ @Override protected boolean convertPortable() { - return true; + return !(cfg.isKeepPortableInStore() && ctx.config().getMarshaller() instanceof PortableMarshaller); } } http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableInputStream.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableInputStream.java deleted file mode 100644 index 501517b..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableInputStream.java +++ /dev/null @@ -1,168 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.portable; - -/** - * Portable input stream. - */ -public interface GridPortableInputStream extends GridPortableStream { - /** - * Read byte value. - * - * @return Byte value. - */ - public byte readByte(); - - /** - * Read byte array. - * - * @param cnt Expected item count. - * @return Byte array. - */ - public byte[] readByteArray(int cnt); - - /** - * Reads {@code cnt} of bytes into byte array. - * - * @param arr Expected item count. - * @param off offset - * @param cnt number of bytes to read. - * @return actual length read. - */ - public int read(byte[] arr, int off, int cnt); - - /** - * Read boolean value. - * - * @return Boolean value. - */ - public boolean readBoolean(); - - /** - * Read boolean array. - * - * @param cnt Expected item count. - * @return Boolean array. - */ - public boolean[] readBooleanArray(int cnt); - - /** - * Read short value. - * - * @return Short value. - */ - public short readShort(); - - /** - * Read short array. - * - * @param cnt Expected item count. - * @return Short array. - */ - public short[] readShortArray(int cnt); - - /** - * Read char value. - * - * @return Char value. - */ - public char readChar(); - - /** - * Read char array. - * - * @param cnt Expected item count. - * @return Char array. - */ - public char[] readCharArray(int cnt); - - /** - * Read int value. - * - * @return Int value. - */ - public int readInt(); - - /** - * Read int value at the given position. - * - * @param pos Position. - * @return Value. - */ - public int readInt(int pos); - - /** - * Read int array. - * - * @param cnt Expected item count. - * @return Int array. - */ - public int[] readIntArray(int cnt); - - /** - * Read float value. - * - * @return Float value. - */ - public float readFloat(); - - /** - * Read float array. - * - * @param cnt Expected item count. - * @return Float array. - */ - public float[] readFloatArray(int cnt); - - /** - * Read long value. - * - * @return Long value. - */ - public long readLong(); - - /** - * Read long array. - * - * @param cnt Expected item count. - * @return Long array. - */ - public long[] readLongArray(int cnt); - - /** - * Read double value. - * - * @return Double value. - */ - public double readDouble(); - - /** - * Read double array. - * - * @param cnt Expected item count. - * @return Double array. - */ - public double[] readDoubleArray(int cnt); - - /** - * Gets amount of remaining data in bytes. - * - * @return Remaining data. - */ - public int remaining(); -} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableOutputStream.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableOutputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableOutputStream.java deleted file mode 100644 index 4f23fd1..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableOutputStream.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.portable; - -/** - * Portable output stream. - */ -public interface GridPortableOutputStream extends GridPortableStream, AutoCloseable { - /** - * Write byte value. - * - * @param val Byte value. - */ - public void writeByte(byte val); - - /** - * Write byte array. - * - * @param val Byte array. - */ - public void writeByteArray(byte[] val); - - /** - * Write boolean value. - * - * @param val Boolean value. - */ - public void writeBoolean(boolean val); - - /** - * Write boolean array. - * - * @param val Boolean array. - */ - public void writeBooleanArray(boolean[] val); - - /** - * Write short value. - * - * @param val Short value. - */ - public void writeShort(short val); - - /** - * Write short array. - * - * @param val Short array. - */ - public void writeShortArray(short[] val); - - /** - * Write char value. - * - * @param val Char value. - */ - public void writeChar(char val); - - /** - * Write char array. - * - * @param val Char array. - */ - public void writeCharArray(char[] val); - - /** - * Write int value. - * - * @param val Int value. - */ - public void writeInt(int val); - - /** - * Write int value to the given position. - * - * @param pos Position. - * @param val Value. - */ - public void writeInt(int pos, int val); - - /** - * Write int array. - * - * @param val Int array. - */ - public void writeIntArray(int[] val); - - /** - * Write float value. - * - * @param val Float value. - */ - public void writeFloat(float val); - - /** - * Write float array. - * - * @param val Float array. - */ - public void writeFloatArray(float[] val); - - /** - * Write long value. - * - * @param val Long value. - */ - public void writeLong(long val); - - /** - * Write long array. - * - * @param val Long array. - */ - public void writeLongArray(long[] val); - - /** - * Write double value. - * - * @param val Double value. - */ - public void writeDouble(double val); - - /** - * Write double array. - * - * @param val Double array. - */ - public void writeDoubleArray(double[] val); - - /** - * Write byte array. - * - * @param arr Array. - * @param off Offset. - * @param len Length. - */ - public void write(byte[] arr, int off, int len); - - /** - * Write data from unmanaged memory. - * - * @param addr Address. - * @param cnt Count. - */ - public void write(long addr, int cnt); - - /** - * Close the stream releasing resources. - */ - @Override public void close(); -} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableStream.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableStream.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableStream.java deleted file mode 100644 index 2c3fc78..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableStream.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.portable; - -/** - * Portable stream. - */ -public interface GridPortableStream { - /** - * @return Position. - */ - public int position(); - - /** - * @param pos Position. - */ - public void position(int pos); - - /** - * @return Underlying array. - */ - public byte[] array(); - - /** - * @return Copy of data in the stream. - */ - public byte[] arrayCopy(); - - /** - * @return Offheap pointer if stream is offheap based, otherwise {@code 0}. - */ - public long offheapPointer(); - - /** - * @return {@code True} is stream is array based. - */ - public boolean hasArray(); -} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/package-info.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/package-info.java deleted file mode 100644 index c6e664c..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/package-info.java +++ /dev/null @@ -1,22 +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. - */ - -/** - * - * Portable processor. - */ -package org.apache.ignite.internal.processors.portable; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java new file mode 100644 index 0000000..2a17363 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java @@ -0,0 +1,347 @@ +/* + * 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.marshaller.portable; + +import org.apache.ignite.*; +import org.apache.ignite.internal.portable.*; +import org.apache.ignite.marshaller.*; +import org.apache.ignite.portable.*; + +import org.jetbrains.annotations.*; + +import java.io.*; +import java.sql.*; +import java.util.*; + +/** + * Implementation of {@link org.apache.ignite.marshaller.Marshaller} that lets to serialize and deserialize all objects + * in the portable format. + *

+ * {@code PortableMarshaller} is tested only on Java HotSpot VM on other VMs it could yield unexpected results. + *

+ *

Configuration

+ *

Mandatory

+ * This marshaller has no mandatory configuration parameters. + *

Java Example

+ *
+ * PortableMarshaller marshaller = new PortableMarshaller();
+ *
+ * IgniteConfiguration cfg = new IgniteConfiguration();
+ *
+ * // Override marshaller.
+ * cfg.setMarshaller(marshaller);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * 
+ *

Spring Example

+ * PortableMarshaller can be configured from Spring XML configuration file: + *
+ * <bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" singleton="true">
+ *     ...
+ *     <property name="marshaller">
+ *         <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller">
+ *            ...
+ *         </bean>
+ *     </property>
+ *     ...
+ * </bean>
+ * 
+ *

+ * + *
+ * For information about Spring framework visit www.springframework.org + */ +public class PortableMarshaller extends AbstractMarshaller { + /** Default portable protocol version. */ + public static final PortableProtocolVersion DFLT_PORTABLE_PROTO_VER = PortableProtocolVersion.VER_1_4_0; + + /** Class names. */ + private Collection clsNames; + + /** ID mapper. */ + private PortableIdMapper idMapper; + + /** Serializer. */ + private PortableSerializer serializer; + + /** Types. */ + private Collection typeCfgs; + + /** Use timestamp flag. */ + private boolean useTs = true; + + /** Whether to convert string to bytes using UTF-8 encoding. */ + private boolean convertString = true; + + /** Meta data enabled flag. */ + private boolean metaDataEnabled = true; + + /** Keep deserialized flag. */ + private boolean keepDeserialized = true; + + /** Protocol version. */ + private PortableProtocolVersion protoVer = DFLT_PORTABLE_PROTO_VER; + + /** */ + private GridPortableMarshaller impl; + + /** + * Gets class names. + * + * @return Class names. + */ + public Collection getClassNames() { + return clsNames; + } + + /** + * Sets class names of portable objects explicitly. + * + * @param clsNames Class names. + */ + public void setClassNames(Collection clsNames) { + this.clsNames = new ArrayList<>(clsNames.size()); + + for (String clsName : clsNames) + this.clsNames.add(clsName.trim()); + } + + /** + * Gets ID mapper. + * + * @return ID mapper. + */ + public PortableIdMapper getIdMapper() { + return idMapper; + } + + /** + * Sets ID mapper. + * + * @param idMapper ID mapper. + */ + public void setIdMapper(PortableIdMapper idMapper) { + this.idMapper = idMapper; + } + + /** + * Gets serializer. + * + * @return Serializer. + */ + public PortableSerializer getSerializer() { + return serializer; + } + + /** + * Sets serializer. + * + * @param serializer Serializer. + */ + public void setSerializer(PortableSerializer serializer) { + this.serializer = serializer; + } + + /** + * Gets types configuration. + * + * @return Types configuration. + */ + public Collection getTypeConfigurations() { + return typeCfgs; + } + + /** + * Sets type configurations. + * + * @param typeCfgs Type configurations. + */ + public void setTypeConfigurations(Collection typeCfgs) { + this.typeCfgs = typeCfgs; + } + + /** + * If {@code true} then date values converted to {@link Timestamp} on deserialization. + *

+ * Default value is {@code true}. + * + * @return Flag indicating whether date values converted to {@link Timestamp} during unmarshalling. + */ + public boolean isUseTimestamp() { + return useTs; + } + + /** + * @param useTs Flag indicating whether date values converted to {@link Timestamp} during unmarshalling. + */ + public void setUseTimestamp(boolean useTs) { + this.useTs = useTs; + } + + /** + * Gets strings must be converted to or from bytes using UTF-8 encoding. + *

+ * Default value is {@code true}. + * + * @return Flag indicating whether string must be converted to byte array using UTF-8 encoding. + */ + public boolean isConvertStringToBytes() { + return convertString; + } + + /** + * Sets strings must be converted to or from bytes using UTF-8 encoding. + *

+ * Default value is {@code true}. + * + * @param convertString Flag indicating whether string must be converted to byte array using UTF-8 encoding. + */ + public void setConvertStringToBytes(boolean convertString) { + this.convertString = convertString; + } + + /** + * If {@code true}, meta data will be collected or all types. If you need to override this behaviour for + * some specific type, use {@link PortableTypeConfiguration#setMetaDataEnabled(Boolean)} method. + *

+ * Default value if {@code true}. + * + * @return Whether meta data is collected. + */ + public boolean isMetaDataEnabled() { + return metaDataEnabled; + } + + /** + * @param metaDataEnabled Whether meta data is collected. + */ + public void setMetaDataEnabled(boolean metaDataEnabled) { + this.metaDataEnabled = metaDataEnabled; + } + + /** + * If {@code true}, {@link PortableObject} will cache deserialized instance after + * {@link PortableObject#deserialize()} is called. All consequent calls of this + * method on the same instance of {@link PortableObject} will return that cached + * value without actually deserializing portable object. If you need to override this + * behaviour for some specific type, use {@link PortableTypeConfiguration#setKeepDeserialized(Boolean)} + * method. + *

+ * Default value if {@code true}. + * + * @return Whether deserialized value is kept. + */ + public boolean isKeepDeserialized() { + return keepDeserialized; + } + + /** + * @param keepDeserialized Whether deserialized value is kept. + */ + public void setKeepDeserialized(boolean keepDeserialized) { + this.keepDeserialized = keepDeserialized; + } + + /** + * Gets portable protocol version. + *

+ * Defaults to {@link #DFLT_PORTABLE_PROTO_VER}. + * + * @return Portable protocol version. + */ + public PortableProtocolVersion getProtocolVersion() { + return protoVer; + } + + /** + * Sets portable protocol version. + *

+ * Defaults to {@link #DFLT_PORTABLE_PROTO_VER}. + * + * @param protoVer Portable protocol version. + */ + public void setProtocolVersion(PortableProtocolVersion protoVer) { + this.protoVer = protoVer; + } + + /** + * Returns currently set {@link MarshallerContext}. + * + * @return Marshaller context. + */ + public MarshallerContext getContext() { + return ctx; + } + + /** + * Sets {@link PortableContext}. + *

+ * @param ctx Portable context. + */ + private void setPortableContext(PortableContext ctx) { + ctx.configure(this); + + impl = new GridPortableMarshaller(ctx); + } + + /** {@inheritDoc} */ + @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException { + return impl.marshal(obj, 0); + } + + /** {@inheritDoc} */ + @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException { + byte[] arr = marshal(obj); + + try { + out.write(arr); + } + catch (IOException e) { + throw new PortableException("Failed to marshal the object: " + obj, e); + } + } + + /** {@inheritDoc} */ + @Override public T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + return impl.deserialize(bytes, clsLdr); + } + + /** {@inheritDoc} */ + @Override public T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + + byte[] arr = new byte[4096]; + int cnt; + + // we have to fully read the InputStream because GridPortableMarshaller requires support of a method that + // returns number of bytes remaining. + try { + while ((cnt = in.read(arr)) != -1) + buffer.write(arr, 0, cnt); + + buffer.flush(); + + return impl.deserialize(buffer.toByteArray(), clsLdr); + } + catch (IOException e) { + throw new PortableException("Failed to unmarshal the object from InputStream", e); + } + } +} + http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/marshaller/portable/package-info.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/package-info.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/package-info.java new file mode 100644 index 0000000..90cc5e6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains portable marshaller API classes. + */ +package org.apache.ignite.marshaller.portable; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java new file mode 100644 index 0000000..a899c46 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java @@ -0,0 +1,138 @@ +/* + * 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.portable; + +import org.apache.ignite.*; + +import org.jetbrains.annotations.*; + +/** + * Portable object builder. Provides ability to build portable objects dynamically without having class definitions. + *

+ * Here is an example of how a portable object can be built dynamically: + *

+ * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
+ *
+ * builder.setField("fieldA", "A");
+ * builder.setField("fieldB", "B");
+ *
+ * PortableObject portableObj = builder.build();
+ * 
+ * + *

+ * Also builder can be initialized by existing portable object. This allows changing some fields without affecting + * other fields. + *

+ * PortableBuilder builder = Ignition.ignite().portables().builder(person);
+ *
+ * builder.setField("name", "John");
+ *
+ * person = builder.build();
+ * 
+ *

+ * + * If you need to modify nested portable object you can get builder for nested object using + * {@link #getField(String)}, changes made on nested builder will affect parent object, + * for example: + * + *
+ * PortableBuilder personBuilder = grid.portables().createBuilder(personPortableObj);
+ * PortableBuilder addressBuilder = personBuilder.setField("address");
+ *
+ * addressBuilder.setField("city", "New York");
+ *
+ * personPortableObj = personBuilder.build();
+ *
+ * // Should be "New York".
+ * String city = personPortableObj.getField("address").getField("city");
+ * 
+ * + * @see IgnitePortables#builder(int) + * @see IgnitePortables#builder(String) + * @see IgnitePortables#builder(PortableObject) + */ +public interface PortableBuilder { + /** + * Returns value assigned to the specified field. + * If the value is a portable object instance of {@code GridPortableBuilder} will be returned, + * which can be modified. + *

+ * Collections and maps returned from this method are modifiable. + * + * @param name Field name. + * @return Filed value. + */ + public T getField(String name); + + /** + * Sets field value. + * + * @param name Field name. + * @param val Field value (cannot be {@code null}). + * @see PortableObject#metaData() + */ + public PortableBuilder setField(String name, Object val); + + /** + * Sets field value with value type specification. + *

+ * Field type is needed for proper metadata update. + * + * @param name Field name. + * @param val Field value. + * @param type Field type. + * @see PortableObject#metaData() + */ + public PortableBuilder setField(String name, @Nullable T val, Class type); + + /** + * Sets field value. + *

+ * This method should be used if field is portable object. + * + * @param name Field name. + * @param builder Builder for object field. + */ + public PortableBuilder setField(String name, @Nullable PortableBuilder builder); + + /** + * Removes field from this builder. + * + * @param fieldName Field name. + * @return {@code this} instance for chaining. + */ + public PortableBuilder removeField(String fieldName); + + /** + * Sets hash code for resulting portable object returned by {@link #build()} method. + *

+ * If not set {@code 0} is used. + * + * @param hashCode Hash code. + * @return {@code this} instance for chaining. + */ + public PortableBuilder hashCode(int hashCode); + + /** + * Builds portable object. + * + * @return Portable object. + * @throws PortableException In case of error. + */ + public PortableObject build() throws PortableException; +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java new file mode 100644 index 0000000..62ae901 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java @@ -0,0 +1,58 @@ +/* + * 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.portable; + +import org.apache.ignite.*; + +import org.jetbrains.annotations.*; + +/** + * Exception indicating portable object serialization error. + */ +public class PortableException extends IgniteException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Creates portable exception with error message. + * + * @param msg Error message. + */ + public PortableException(String msg) { + super(msg); + } + + /** + * Creates portable exception with {@link Throwable} as a cause. + * + * @param cause Cause. + */ + public PortableException(Throwable cause) { + super(cause); + } + + /** + * Creates portable exception with error message and {@link Throwable} as a cause. + * + * @param msg Error message. + * @param cause Cause. + */ + public PortableException(String msg, @Nullable Throwable cause) { + super(msg, cause); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java new file mode 100644 index 0000000..9502a86 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java @@ -0,0 +1,56 @@ +/* + * 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.portable; + +import org.apache.ignite.marshaller.portable.*; + +/** + * Type and field ID mapper for portable objects. Ignite never writes full + * strings for field or type names. Instead, for performance reasons, Ignite + * writes integer hash codes for type and field names. It has been tested that + * hash code conflicts for the type names or the field names + * within the same type are virtually non-existent and, to gain performance, it is safe + * to work with hash codes. For the cases when hash codes for different types or fields + * actually do collide {@code PortableIdMapper} allows to override the automatically + * generated hash code IDs for the type and field names. + *

+ * Portable ID mapper can be configured for all portable objects via {@link PortableMarshaller#getIdMapper()} method, + * or for a specific portable type via {@link PortableTypeConfiguration#getIdMapper()} method. + */ +public interface PortableIdMapper { + /** + * Gets type ID for provided class name. + *

+ * If {@code 0} is returned, hash code of class simple name will be used. + * + * @param clsName Class name. + * @return Type ID. + */ + public int typeId(String clsName); + + /** + * Gets ID for provided field. + *

+ * If {@code 0} is returned, hash code of field name will be used. + * + * @param typeId Type ID. + * @param fieldName Field name. + * @return Field ID. + */ + public int fieldId(int typeId, String fieldName); +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java new file mode 100644 index 0000000..533d453 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java @@ -0,0 +1,58 @@ +/* + * 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.portable; + +import org.jetbrains.annotations.*; + +/** + * Exception indicating that class needed for deserialization of portable object does not exist. + *

+ * Thrown from {@link PortableObject#deserialize()} method. + */ +public class PortableInvalidClassException extends PortableException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Creates invalid class exception with error message. + * + * @param msg Error message. + */ + public PortableInvalidClassException(String msg) { + super(msg); + } + + /** + * Creates invalid class exception with {@link Throwable} as a cause. + * + * @param cause Cause. + */ + public PortableInvalidClassException(Throwable cause) { + super(cause); + } + + /** + * Creates invalid class exception with error message and {@link Throwable} as a cause. + * + * @param msg Error message. + * @param cause Cause. + */ + public PortableInvalidClassException(String msg, @Nullable Throwable cause) { + super(msg, cause); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java new file mode 100644 index 0000000..3ae2bd7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java @@ -0,0 +1,48 @@ +/* + * 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.portable; + +/** + * Interface that allows to implement custom serialization + * logic for portable objects. Portable objects are not required + * to implement this interface, in which case Ignite will automatically + * serialize portable objects using reflection. + *

+ * This interface, in a way, is analogous to {@link java.io.Externalizable} + * interface, which allows users to override default serialization logic, + * usually for performance reasons. The only difference here is that portable + * serialization is already very fast and implementing custom serialization + * logic for portables does not provide significant performance gains. + */ +public interface PortableMarshalAware { + /** + * Writes fields to provided writer. + * + * @param writer Portable object writer. + * @throws PortableException In case of error. + */ + public void writePortable(PortableWriter writer) throws PortableException; + + /** + * Reads fields from provided reader. + * + * @param reader Portable object reader. + * @throws PortableException In case of error. + */ + public void readPortable(PortableReader reader) throws PortableException; +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java new file mode 100644 index 0000000..7697299 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java @@ -0,0 +1,63 @@ +/* + * 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.portable; + +import org.apache.ignite.*; + +import org.jetbrains.annotations.*; + +import java.util.*; + +/** + * Portable type meta data. Metadata for portable types can be accessed from any of the + * {@link IgnitePortables#metadata(String)} methods. + * Having metadata also allows for proper formatting of {@code PortableObject#toString()} method, + * even when portable objects are kept in binary format only, which may be necessary for audit reasons. + */ +public interface PortableMetadata { + /** + * Gets portable type name. + * + * @return Portable type name. + */ + public String typeName(); + + /** + * Gets collection of all field names for this portable type. + * + * @return Collection of all field names for this portable type. + */ + public Collection fields(); + + /** + * Gets name of the field type for a given field. + * + * @param fieldName Field name. + * @return Field type name. + */ + @Nullable public String fieldTypeName(String fieldName); + + /** + * Portable 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. + * + * @return Affinity key field name. + */ + @Nullable public String affinityKeyFieldName(); +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java new file mode 100644 index 0000000..c22892d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java @@ -0,0 +1,153 @@ +/* + * 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.portable; + +import org.apache.ignite.*; +import org.apache.ignite.marshaller.portable.*; + +import org.jetbrains.annotations.*; + +import java.io.*; +import java.util.*; + +/** + * Wrapper for portable object in portable binary format. Once an object is defined as portable, + * Ignite will always store it in memory in the portable (i.e. binary) format. + * User can choose to work either with the portable format or with the deserialized form + * (assuming that class definitions are present in the classpath). + *

+ * NOTE: user does not need to (and should not) implement this interface directly. + *

+ * To work with the portable format directly, user should create a cache projection + * over {@code PortableObject} class and then retrieve individual fields as needed: + *

+ * IgniteCache<PortableObject, PortableObject> prj = cache.withKeepPortable();
+ *
+ * // Convert instance of MyKey to portable format.
+ * // We could also use GridPortableBuilder to create the key in portable format directly.
+ * PortableObject key = grid.portables().toPortable(new MyKey());
+ *
+ * PortableObject val = prj.get(key);
+ *
+ * String field = val.field("myFieldName");
+ * 
+ * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized + * typed objects at all times. In this case we do incur the deserialization cost. However, if + * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access + * and will cache the deserialized object, so it does not have to be deserialized again: + *
+ * IgniteCache<MyKey.class, MyValue.class> cache = grid.cache(null);
+ *
+ * MyValue val = cache.get(new MyKey());
+ *
+ * // Normal java getter.
+ * String fieldVal = val.getMyFieldName();
+ * 
+ *

Working With Maps and Collections

+ * All maps and collections in the portable objects are serialized automatically. When working + * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most + * adequate collection or map in either language. For example, {@link ArrayList} in Java will become + * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap} + * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary} + * in C#, etc. + *

Dynamic Structure Changes

+ * Since objects are always cached in the portable binary format, server does not need to + * be aware of the class definitions. Moreover, if class definitions are not present or not + * used on the server, then clients can continuously change the structure of the portable + * objects without having to restart the cluster. For example, if one client stores a + * certain class with fields A and B, and another client stores the same class with + * fields B and C, then the server-side portable object will have the fields A, B, and C. + * As the structure of a portable object changes, the new fields become available for SQL queries + * automatically. + *

Building Portable Objects

+ * Ignite comes with {@link PortableBuilder} which allows to build portable objects dynamically: + *
+ * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
+ *
+ * builder.setField("fieldA", "A");
+ * builder.setField("fieldB", "B");
+ *
+ * PortableObject portableObj = builder.build();
+ * 
+ * For the cases when class definition is present + * in the class path, it is also possible to populate a standard POJO and then + * convert it to portable format, like so: + *
+ * MyObject obj = new MyObject();
+ *
+ * obj.setFieldA("A");
+ * obj.setFieldB(123);
+ *
+ * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
+ * 
+ *

Portable Metadata

+ * Even though Ignite portable protocol only works with hash codes for type and field names + * to achieve better performance, Ignite provides metadata for all portable types which + * can be queried ar runtime via any of the {@link IgnitePortables#metadata(Class)} + * methods. Having metadata also allows for proper formatting of {@code PortableObject.toString()} method, + * even when portable objects are kept in binary format only, which may be necessary for audit reasons. + */ +public interface PortableObject extends Serializable, Cloneable { + /** + * Gets portable object type ID. + * + * @return Type ID. + */ + public int typeId(); + + /** + * Gets meta data for this portable object. + * + * @return Meta data. + * @throws PortableException In case of error. + */ + @Nullable public PortableMetadata metaData() throws PortableException; + + /** + * Gets field value. + * + * @param fieldName Field name. + * @return Field value. + * @throws PortableException In case of any other error. + */ + @Nullable public F field(String fieldName) throws PortableException; + + /** + * Checks whether field is set. + * + * @param fieldName Field name. + * @return {@code true} if field is set. + */ + public boolean hasField(String fieldName); + + /** + * Gets fully deserialized instance of portable object. + * + * @return Fully deserialized instance of portable object. + * @throws PortableInvalidClassException If class doesn't exist. + * @throws PortableException In case of any other error. + */ + @Nullable public T deserialize() throws PortableException; + + /** + * Copies this portable object. + * + * @return Copy of this portable object. + */ + public PortableObject clone() throws CloneNotSupportedException; +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableProtocolVersion.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableProtocolVersion.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableProtocolVersion.java new file mode 100644 index 0000000..5764560 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableProtocolVersion.java @@ -0,0 +1,41 @@ +/* + * 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.portable; + +import org.jetbrains.annotations.*; + +/** + * Portable protocol version. + */ +public enum PortableProtocolVersion { + /** Ignite 1.4.0 release. */ + VER_1_4_0; + + /** Enumerated values. */ + private static final PortableProtocolVersion[] VALS = values(); + + /** + * Efficiently gets enumerated value from its ordinal. + * + * @param ord Ordinal value. + * @return Enumerated value or {@code null} if ordinal out of range. + */ + @Nullable public static PortableProtocolVersion fromOrdinal(int ord) { + return ord >= 0 && ord < VALS.length ? VALS[ord] : null; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableRawReader.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableRawReader.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableRawReader.java new file mode 100644 index 0000000..a704570 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableRawReader.java @@ -0,0 +1,233 @@ +/* + * 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.portable; + +import org.jetbrains.annotations.*; + +import java.math.*; +import java.sql.*; +import java.util.*; +import java.util.Date; + +/** + * Raw reader for portable objects. Raw reader does not use field name hash codes, therefore, + * making the format even more compact. However, if the raw reader is used, + * dynamic structure changes to the portable objects are not supported. + */ +public interface PortableRawReader { + /** + * @return Byte value. + * @throws PortableException In case of error. + */ + public byte readByte() throws PortableException; + + /** + * @return Short value. + * @throws PortableException In case of error. + */ + public short readShort() throws PortableException; + + /** + * @return Integer value. + * @throws PortableException In case of error. + */ + public int readInt() throws PortableException; + + /** + * @return Long value. + * @throws PortableException In case of error. + */ + public long readLong() throws PortableException; + + /** + * @return Float value. + * @throws PortableException In case of error. + */ + public float readFloat() throws PortableException; + + /** + * @return Double value. + * @throws PortableException In case of error. + */ + public double readDouble() throws PortableException; + + /** + * @return Char value. + * @throws PortableException In case of error. + */ + public char readChar() throws PortableException; + + /** + * @return Boolean value. + * @throws PortableException In case of error. + */ + public boolean readBoolean() throws PortableException; + + /** + * @return Decimal value. + * @throws PortableException In case of error. + */ + @Nullable public BigDecimal readDecimal() throws PortableException; + + /** + * @return String value. + * @throws PortableException In case of error. + */ + @Nullable public String readString() throws PortableException; + + /** + * @return UUID. + * @throws PortableException In case of error. + */ + @Nullable public UUID readUuid() throws PortableException; + + /** + * @return Date. + * @throws PortableException In case of error. + */ + @Nullable public Date readDate() throws PortableException; + + /** + * @return Timestamp. + * @throws PortableException In case of error. + */ + @Nullable public Timestamp readTimestamp() throws PortableException; + + /** + * @return Object. + * @throws PortableException In case of error. + */ + @Nullable public T readObject() throws PortableException; + + /** + * @return Byte array. + * @throws PortableException In case of error. + */ + @Nullable public byte[] readByteArray() throws PortableException; + + /** + * @return Short array. + * @throws PortableException In case of error. + */ + @Nullable public short[] readShortArray() throws PortableException; + + /** + * @return Integer array. + * @throws PortableException In case of error. + */ + @Nullable public int[] readIntArray() throws PortableException; + + /** + * @return Long array. + * @throws PortableException In case of error. + */ + @Nullable public long[] readLongArray() throws PortableException; + + /** + * @return Float array. + * @throws PortableException In case of error. + */ + @Nullable public float[] readFloatArray() throws PortableException; + + /** + * @return Byte array. + * @throws PortableException In case of error. + */ + @Nullable public double[] readDoubleArray() throws PortableException; + + /** + * @return Char array. + * @throws PortableException In case of error. + */ + @Nullable public char[] readCharArray() throws PortableException; + + /** + * @return Boolean array. + * @throws PortableException In case of error. + */ + @Nullable public boolean[] readBooleanArray() throws PortableException; + + /** + * @return Decimal array. + * @throws PortableException In case of error. + */ + @Nullable public BigDecimal[] readDecimalArray() throws PortableException; + + /** + * @return String array. + * @throws PortableException In case of error. + */ + @Nullable public String[] readStringArray() throws PortableException; + + /** + * @return UUID array. + * @throws PortableException In case of error. + */ + @Nullable public UUID[] readUuidArray() throws PortableException; + + /** + * @return Date array. + * @throws PortableException In case of error. + */ + @Nullable public Date[] readDateArray() throws PortableException; + + /** + * @return Object array. + * @throws PortableException In case of error. + */ + @Nullable public Object[] readObjectArray() throws PortableException; + + /** + * @return Collection. + * @throws PortableException In case of error. + */ + @Nullable public Collection readCollection() throws PortableException; + + /** + * @param colCls Collection class. + * @return Collection. + * @throws PortableException In case of error. + */ + @Nullable public Collection readCollection(Class> colCls) + throws PortableException; + + /** + * @return Map. + * @throws PortableException In case of error. + */ + @Nullable public Map readMap() throws PortableException; + + /** + * @param mapCls Map class. + * @return Map. + * @throws PortableException In case of error. + */ + @Nullable public Map readMap(Class> mapCls) throws PortableException; + + /** + * @return Value. + * @throws PortableException In case of error. + */ + @Nullable public > T readEnum() throws PortableException; + + /** + * @return Value. + * @throws PortableException In case of error. + */ + @Nullable public > T[] readEnumArray() throws PortableException; +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableRawWriter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableRawWriter.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableRawWriter.java new file mode 100644 index 0000000..e6efee5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableRawWriter.java @@ -0,0 +1,218 @@ +/* + * 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.portable; + +import org.jetbrains.annotations.*; + +import java.math.*; +import java.sql.*; +import java.util.*; +import java.util.Date; + +/** + * Raw writer for portable object. Raw writer does not write field name hash codes, therefore, + * making the format even more compact. However, if the raw writer is used, + * dynamic structure changes to the portable objects are not supported. + */ +public interface PortableRawWriter { + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeByte(byte val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeShort(short val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeInt(int val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeLong(long val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeFloat(float val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeDouble(double val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeChar(char val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeBoolean(boolean val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeDecimal(@Nullable BigDecimal val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeString(@Nullable String val) throws PortableException; + + /** + * @param val UUID to write. + * @throws PortableException In case of error. + */ + public void writeUuid(@Nullable UUID val) throws PortableException; + + /** + * @param val Date to write. + * @throws PortableException In case of error. + */ + public void writeDate(@Nullable Date val) throws PortableException; + + /** + * @param val Timestamp to write. + * @throws PortableException In case of error. + */ + public void writeTimestamp(@Nullable Timestamp val) throws PortableException; + + /** + * @param obj Value to write. + * @throws PortableException In case of error. + */ + public void writeObject(@Nullable Object obj) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeByteArray(@Nullable byte[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeShortArray(@Nullable short[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeIntArray(@Nullable int[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeLongArray(@Nullable long[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeFloatArray(@Nullable float[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeDoubleArray(@Nullable double[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeCharArray(@Nullable char[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeBooleanArray(@Nullable boolean[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeDecimalArray(@Nullable BigDecimal[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeStringArray(@Nullable String[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeUuidArray(@Nullable UUID[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeDateArray(@Nullable Date[] val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public void writeObjectArray(@Nullable Object[] val) throws PortableException; + + /** + * @param col Collection to write. + * @throws PortableException In case of error. + */ + public void writeCollection(@Nullable Collection col) throws PortableException; + + /** + * @param map Map to write. + * @throws PortableException In case of error. + */ + public void writeMap(@Nullable Map map) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public > void writeEnum(T val) throws PortableException; + + /** + * @param val Value to write. + * @throws PortableException In case of error. + */ + public > void writeEnumArray(T[] val) throws PortableException; +} http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java new file mode 100644 index 0000000..82dca0c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java @@ -0,0 +1,283 @@ +/* + * 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.portable; + +import org.jetbrains.annotations.*; + +import java.math.*; +import java.sql.*; +import java.util.*; +import java.util.Date; + +/** + * Reader for portable objects used in {@link PortableMarshalAware} implementations. + * Useful for the cases when user wants a fine-grained control over serialization. + *

+ * Note that Ignite never writes full strings for field or type names. Instead, + * for performance reasons, Ignite writes integer hash codes for type and field names. + * It has been tested that hash code conflicts for the type names or the field names + * within the same type are virtually non-existent and, to gain performance, it is safe + * to work with hash codes. For the cases when hash codes for different types or fields + * actually do collide, Ignite provides {@link PortableIdMapper} which + * allows to override the automatically generated hash code IDs for the type and field names. + */ +public interface PortableReader { + /** + * @param fieldName Field name. + * @return Byte value. + * @throws PortableException In case of error. + */ + public byte readByte(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Short value. + * @throws PortableException In case of error. + */ + public short readShort(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Integer value. + * @throws PortableException In case of error. + */ + public int readInt(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Long value. + * @throws PortableException In case of error. + */ + public long readLong(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @throws PortableException In case of error. + * @return Float value. + */ + public float readFloat(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Double value. + * @throws PortableException In case of error. + */ + public double readDouble(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Char value. + * @throws PortableException In case of error. + */ + public char readChar(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Boolean value. + * @throws PortableException In case of error. + */ + public boolean readBoolean(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Decimal value. + * @throws PortableException In case of error. + */ + @Nullable public BigDecimal readDecimal(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return String value. + * @throws PortableException In case of error. + */ + @Nullable public String readString(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return UUID. + * @throws PortableException In case of error. + */ + @Nullable public UUID readUuid(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Date. + * @throws PortableException In case of error. + */ + @Nullable public Date readDate(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Timestamp. + * @throws PortableException In case of error. + */ + @Nullable public Timestamp readTimestamp(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Object. + * @throws PortableException In case of error. + */ + @Nullable public T readObject(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Byte array. + * @throws PortableException In case of error. + */ + @Nullable public byte[] readByteArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Short array. + * @throws PortableException In case of error. + */ + @Nullable public short[] readShortArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Integer array. + * @throws PortableException In case of error. + */ + @Nullable public int[] readIntArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Long array. + * @throws PortableException In case of error. + */ + @Nullable public long[] readLongArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Float array. + * @throws PortableException In case of error. + */ + @Nullable public float[] readFloatArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Byte array. + * @throws PortableException In case of error. + */ + @Nullable public double[] readDoubleArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Char array. + * @throws PortableException In case of error. + */ + @Nullable public char[] readCharArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Boolean array. + * @throws PortableException In case of error. + */ + @Nullable public boolean[] readBooleanArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Decimal array. + * @throws PortableException In case of error. + */ + @Nullable public BigDecimal[] readDecimalArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return String array. + * @throws PortableException In case of error. + */ + @Nullable public String[] readStringArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return UUID array. + * @throws PortableException In case of error. + */ + @Nullable public UUID[] readUuidArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Date array. + * @throws PortableException In case of error. + */ + @Nullable public Date[] readDateArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Object array. + * @throws PortableException In case of error. + */ + @Nullable public Object[] readObjectArray(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Collection. + * @throws PortableException In case of error. + */ + @Nullable public Collection readCollection(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @param colCls Collection class. + * @return Collection. + * @throws PortableException In case of error. + */ + @Nullable public Collection readCollection(String fieldName, Class> colCls) + throws PortableException; + + /** + * @param fieldName Field name. + * @return Map. + * @throws PortableException In case of error. + */ + @Nullable public Map readMap(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @param mapCls Map class. + * @return Map. + * @throws PortableException In case of error. + */ + @Nullable public Map readMap(String fieldName, Class> mapCls) + throws PortableException; + + /** + * @param fieldName Field name. + * @return Value. + * @throws PortableException In case of error. + */ + @Nullable public > T readEnum(String fieldName) throws PortableException; + + /** + * @param fieldName Field name. + * @return Value. + * @throws PortableException In case of error. + */ + @Nullable public > T[] readEnumArray(String fieldName) throws PortableException; + + /** + * Gets raw reader. Raw reader does not use field name hash codes, therefore, + * making the format even more compact. However, if the raw reader is used, + * dynamic structure changes to the portable objects are not supported. + * + * @return Raw reader. + */ + public PortableRawReader rawReader(); +}