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 ED38317707 for ; Fri, 4 Sep 2015 10:36:43 +0000 (UTC) Received: (qmail 46020 invoked by uid 500); 4 Sep 2015 10:36:43 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 45929 invoked by uid 500); 4 Sep 2015 10:36:43 -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 45509 invoked by uid 99); 4 Sep 2015 10:36:43 -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, 04 Sep 2015 10:36:43 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 6BA17E0F7D; Fri, 4 Sep 2015 10:36:43 +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, 04 Sep 2015 10:36:53 -0000 Message-Id: <9e03aa91ff9e41989aa1f3cc1f136e1b@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [11/50] [abbrv] ignite git commit: ignite-1273: added ability to modify arrays returned from PortableBuilder and fixed cyclic references processing for arrays and collections in PortableMarshaller http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java deleted file mode 100644 index eed8121..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMap.java +++ /dev/null @@ -1,218 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.portable; - -import java.util.AbstractMap; -import java.util.AbstractSet; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.Set; -import org.jetbrains.annotations.Nullable; - -/** - * - */ -class PortableLazyMap extends AbstractMap implements PortableBuilderSerializationAware { - /** */ - private final PortableBuilderReader reader; - - /** */ - private final int off; - - /** */ - private Map delegate; - - /** - * @param reader Reader. - * @param off Offset. - */ - private PortableLazyMap(PortableBuilderReader reader, int off) { - this.reader = reader; - this.off = off; - } - - /** - * @param reader Reader. - * @return PortableLazyMap. - */ - @Nullable public static PortableLazyMap parseMap(PortableBuilderReader reader) { - int off = reader.position() - 1; - - int size = reader.readInt(); - - reader.skip(1); // map type. - - for (int i = 0; i < size; i++) { - reader.skipValue(); // skip key - reader.skipValue(); // skip value - } - - return new PortableLazyMap(reader, off); - } - - /** - * - */ - private void ensureDelegateInit() { - if (delegate == null) { - int size = reader.readIntAbsolute(off + 1); - - reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */); - - delegate = new LinkedHashMap<>(); - - for (int i = 0; i < size; i++) - delegate.put(PortableUtils.unwrapLazy(reader.parseValue()), reader.parseValue()); - } - } - - /** {@inheritDoc} */ - @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) { - if (delegate == null) { - int size = reader.readIntAbsolute(off + 1); - - int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */; - writer.write(reader.array(), off, hdrSize); - - reader.position(off + hdrSize); - - for (int i = 0; i < size; i++) { - ctx.writeValue(writer, reader.parseValue()); // key - ctx.writeValue(writer, reader.parseValue()); // value - } - } - else { - writer.writeByte(GridPortableMarshaller.MAP); - writer.writeInt(delegate.size()); - - byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */]; - - writer.writeByte(colType); - - for (Entry entry : delegate.entrySet()) { - ctx.writeValue(writer, entry.getKey()); - ctx.writeValue(writer, entry.getValue()); - } - } - } - - /** {@inheritDoc} */ - @Override public int size() { - if (delegate == null) - return reader.readIntAbsolute(off + 1); - - return delegate.size(); - } - - /** {@inheritDoc} */ - @Override public boolean containsKey(Object key) { - ensureDelegateInit(); - - return delegate.containsKey(key); - } - - /** {@inheritDoc} */ - @Override public boolean containsValue(Object val) { - return values().contains(val); - } - - /** {@inheritDoc} */ - @Override public Set keySet() { - ensureDelegateInit(); - - return delegate.keySet(); - } - - /** {@inheritDoc} */ - @Override public void clear() { - if (delegate == null) - delegate = new LinkedHashMap<>(); - else - delegate.clear(); - } - - /** {@inheritDoc} */ - @Override public Object get(Object key) { - ensureDelegateInit(); - - return PortableUtils.unwrapLazy(delegate.get(key)); - } - - /** {@inheritDoc} */ - @Override public Object put(Object key, Object val) { - ensureDelegateInit(); - - return PortableUtils.unwrapLazy(delegate.put(key, val)); - } - - /** {@inheritDoc} */ - @Override public Object remove(Object key) { - ensureDelegateInit(); - - return PortableUtils.unwrapLazy(delegate.remove(key)); - } - - /** {@inheritDoc} */ - @Override public Set> entrySet() { - ensureDelegateInit(); - - return new AbstractSet>() { - @Override public boolean contains(Object o) { - throw new UnsupportedOperationException(); - } - - @Override public Iterator> iterator() { - return new Iterator>() { - /** */ - private final Iterator> itr = delegate.entrySet().iterator(); - - @Override public boolean hasNext() { - return itr.hasNext(); - } - - @Override public Entry next() { - Entry res = itr.next(); - - final Object val = res.getValue(); - - if (val instanceof PortableLazyValue) { - return new SimpleEntry(res.getKey(), val) { - private static final long serialVersionUID = 0L; - - @Override public Object getValue() { - return ((PortableLazyValue)val).value(); - } - }; - } - - return res; - } - - @Override public void remove() { - itr.remove(); - } - }; - } - - @Override public int size() { - return delegate.size(); - } - }; - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java deleted file mode 100644 index 1970d21..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyMapEntry.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.portable; - -import java.util.Map; - -/** - * - */ -class PortableLazyMapEntry implements Map.Entry, PortableBuilderSerializationAware { - /** */ - private final Object key; - - /** */ - private Object val; - - /** - * @param reader GridMutablePortableReader - */ - PortableLazyMapEntry(PortableBuilderReader reader) { - key = reader.parseValue(); - val = reader.parseValue(); - } - - /** {@inheritDoc} */ - @Override public Object getKey() { - return PortableUtils.unwrapLazy(key); - } - - /** {@inheritDoc} */ - @Override public Object getValue() { - return PortableUtils.unwrapLazy(val); - } - - /** {@inheritDoc} */ - @Override public Object setValue(Object val) { - Object res = getValue(); - - this.val = val; - - return res; - } - - /** {@inheritDoc} */ - @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) { - writer.writeByte(GridPortableMarshaller.MAP_ENTRY); - - ctx.writeValue(writer, key); - ctx.writeValue(writer, val); - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java deleted file mode 100644 index 3e1dc92..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazySet.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.portable; - -import java.util.Collection; -import java.util.Set; -import org.apache.ignite.internal.util.typedef.internal.U; - -/** - * - */ -class PortableLazySet extends PortableAbstractLazyValue { - /** */ - private final int off; - - /** - * @param reader Reader. - * @param size Size. - */ - PortableLazySet(PortableBuilderReader reader, int size) { - super(reader, reader.position() - 1); - - off = reader.position() - 1/* flag */ - 4/* size */ - 1/* col type */; - - assert size >= 0; - - for (int i = 0; i < size; i++) - reader.skipValue(); - } - - /** {@inheritDoc} */ - @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) { - if (val == null) { - int size = reader.readIntAbsolute(off + 1); - - int hdrSize = 1 /* flag */ + 4 /* size */ + 1 /* col type */; - writer.write(reader.array(), off, hdrSize); - - reader.position(off + hdrSize); - - for (int i = 0; i < size; i++) { - Object o = reader.parseValue(); - - ctx.writeValue(writer, o); - } - } - else { - Collection c = (Collection)val; - - writer.writeByte(GridPortableMarshaller.COL); - writer.writeInt(c.size()); - - byte colType = reader.array()[off + 1 /* flag */ + 4 /* size */]; - writer.writeByte(colType); - - for (Object o : c) - ctx.writeValue(writer, o); - } - } - - /** {@inheritDoc} */ - @Override protected Object init() { - int size = reader.readIntAbsolute(off + 1); - - reader.position(off + 1/* flag */ + 4/* size */ + 1/* col type */); - - Set res = U.newLinkedHashSet(size); - - for (int i = 0; i < size; i++) - res.add(PortableUtils.unwrapLazy(reader.parseValue())); - - return res; - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java deleted file mode 100644 index 43728b7..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableLazyValue.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.portable; - -/** - * - */ -interface PortableLazyValue extends PortableBuilderSerializationAware { - /** - * @return Value. - */ - public Object value(); -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java deleted file mode 100644 index 897e8e8..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectArrayLazyValue.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.portable; - -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.portable.PortableInvalidClassException; - -/** - * - */ -class PortableObjectArrayLazyValue extends PortableAbstractLazyValue { - /** */ - private Object[] lazyValsArr; - - /** */ - private int compTypeId; - - /** */ - private String clsName; - - /** - * @param reader Reader. - */ - protected PortableObjectArrayLazyValue(PortableBuilderReader reader) { - super(reader, reader.position() - 1); - - int typeId = reader.readInt(); - - if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) { - clsName = reader.readString(); - - Class cls; - - try { - // TODO: IGNITE-1272 - Is class loader needed here? - cls = U.forName(reader.readString(), null); - } - catch (ClassNotFoundException e) { - throw new PortableInvalidClassException("Failed to load the class: " + clsName, e); - } - - compTypeId = reader.portableContext().descriptorForClass(cls).typeId(); - } - else { - compTypeId = typeId; - clsName = null; - } - - int size = reader.readInt(); - - lazyValsArr = new Object[size]; - - for (int i = 0; i < size; i++) - lazyValsArr[i] = reader.parseValue(); - } - - /** {@inheritDoc} */ - @Override protected Object init() { - for (int i = 0; i < lazyValsArr.length; i++) { - if (lazyValsArr[i] instanceof PortableLazyValue) - lazyValsArr[i] = ((PortableLazyValue)lazyValsArr[i]).value(); - } - - return lazyValsArr; - } - - /** {@inheritDoc} */ - @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) { - if (clsName == null) - ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, compTypeId); - else - ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, clsName); - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java deleted file mode 100644 index d08d09b..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainLazyValue.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.portable; - -/** - * - */ -class PortablePlainLazyValue extends PortableAbstractLazyValue { - /** */ - protected final int len; - - /** - * @param reader Reader - * @param valOff Offset - * @param len Length. - */ - protected PortablePlainLazyValue(PortableBuilderReader reader, int valOff, int len) { - super(reader, valOff); - - this.len = len; - } - - /** {@inheritDoc} */ - @Override protected Object init() { - return reader.reader().unmarshal(valOff); - } - - /** {@inheritDoc} */ - @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) { - writer.write(reader.array(), valOff, len); - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java deleted file mode 100644 index cfaa04f..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortablePlainPortableObject.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.portable; - -import org.apache.ignite.portable.PortableObject; - -/** - * - */ -public class PortablePlainPortableObject implements PortableLazyValue { - /** */ - private final PortableObject portableObj; - - /** - * @param portableObj Portable object. - */ - public PortablePlainPortableObject(PortableObject portableObj) { - this.portableObj = portableObj; - } - - /** {@inheritDoc} */ - @Override public Object value() { - return portableObj; - } - - /** {@inheritDoc} */ - @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) { - PortableObject val = portableObj; - - if (val instanceof PortableObjectOffheapImpl) - val = ((PortableObjectOffheapImpl)val).heapCopy(); - - writer.doWritePortableObject((PortableObjectImpl)val); - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java index f702e06..83ccb65 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java @@ -156,7 +156,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @param start Start. * @param ldr Class loader. */ - PortableReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) { + public PortableReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) { this(ctx, new PortableHeapInputStream(arr), start, ldr, new PortableReaderContext()); } @@ -256,7 +256,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @return Unmarshalled value. * @throws PortableException In case of error. */ - Object unmarshal(int offset) throws PortableException { + public Object unmarshal(int offset) throws PortableException { off = offset; return off >= 0 ? unmarshal(false) : null; @@ -586,6 +586,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != BYTE_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -609,6 +612,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != SHORT_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -632,6 +638,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != INT_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -655,6 +664,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != LONG_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -678,6 +690,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != FLOAT_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -701,6 +716,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != DOUBLE_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -724,6 +742,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != CHAR_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -747,6 +768,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != BOOLEAN_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -770,6 +794,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != DECIMAL_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -793,6 +820,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != STRING_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -816,6 +846,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != UUID_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -839,6 +872,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != DATE_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -862,6 +898,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != OBJ_ARR) throw new PortableException("Invalid flag value: " + flag); @@ -887,6 +926,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != COL) throw new PortableException("Invalid flag value: " + flag); @@ -912,6 +954,9 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != MAP) throw new PortableException("Invalid flag value: " + flag); @@ -935,10 +980,13 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx if (flag == NULL) return null; + if (flag == HANDLE) + return readHandleField(); + if (flag != MAP_ENTRY) throw new PortableException("Invalid flag value: " + flag); - return new GridMapEntry<>(doReadObject(false), doReadObject(false)); + return doReadMapEntry(false, true); } else return null; @@ -1059,6 +1107,33 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx rCtx.setObjectHandler(start, obj); } + /** + * @param obj Object. + * @param pos Position. + */ + void setHandler(Object obj, int pos) { + rCtx.setObjectHandler(pos, obj); + } + + /** + * Recreating field value from a handle. + * + * @param Field type. + * @return Field. + */ + private T readHandleField() { + int handle = (off - 1) - doReadInt(false); + + Object obj = rCtx.getObjectByHandle(handle); + + if (obj == null) { + off = handle; + + obj = doReadObject(false); + } + + return (T)obj; + } /** {@inheritDoc} */ @Override public byte readByte(String fieldName) throws PortableException { Byte val = readByte(fieldId(fieldName)); @@ -1676,7 +1751,7 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx else po = in.offheapPointer() > 0 ? new PortableObjectOffheapImpl(ctx, in.offheapPointer(), start, - in.remaining() + in.position()) + in.remaining() + in.position()) : new PortableObjectImpl(ctx, in.array(), start); rCtx.setPortableHandler(start, po); @@ -1805,7 +1880,6 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx return obj; - default: throw new PortableException("Invalid flag value: " + flag); } @@ -2306,12 +2380,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @return Value. */ private byte[] doReadByteArray(boolean raw) { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); in.position(raw ? rawOff : off); byte[] arr = in.readByteArray(len); + setHandler(arr, hPos); + if (raw) rawOff += len; else @@ -2325,12 +2403,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @return Value. */ private short[] doReadShortArray(boolean raw) { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); in.position(raw ? rawOff : off); short[] arr = in.readShortArray(len); + setHandler(arr, hPos); + int bytes = len << 1; if (raw) @@ -2346,12 +2428,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @return Value. */ private int[] doReadIntArray(boolean raw) { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); in.position(raw ? rawOff : off); int[] arr = in.readIntArray(len); + setHandler(arr, hPos); + int bytes = len << 2; if (raw) @@ -2367,12 +2453,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @return Value. */ private long[] doReadLongArray(boolean raw) { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); in.position(raw ? rawOff : off); long[] arr = in.readLongArray(len); + setHandler(arr, hPos); + int bytes = len << 3; if (raw) @@ -2388,12 +2478,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @return Value. */ private float[] doReadFloatArray(boolean raw) { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); in.position(raw ? rawOff : off); float[] arr = in.readFloatArray(len); + setHandler(arr, hPos); + int bytes = len << 2; if (raw) @@ -2409,12 +2503,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @return Value. */ private double[] doReadDoubleArray(boolean raw) { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); in.position(raw ? rawOff : off); double[] arr = in.readDoubleArray(len); + setHandler(arr, hPos); + int bytes = len << 3; if (raw) @@ -2430,12 +2528,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @return Value. */ private char[] doReadCharArray(boolean raw) { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); in.position(raw ? rawOff : off); char[] arr = in.readCharArray(len); + setHandler(arr, hPos); + int bytes = len << 1; if (raw) @@ -2451,12 +2553,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @return Value. */ private boolean[] doReadBooleanArray(boolean raw) { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); in.position(raw ? rawOff : off); boolean[] arr = in.readBooleanArray(len); + setHandler(arr, hPos); + if (raw) rawOff += len; else @@ -2471,10 +2577,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @throws PortableException In case of error. */ private BigDecimal[] doReadDecimalArray(boolean raw) throws PortableException { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); BigDecimal[] arr = new BigDecimal[len]; + setHandler(arr, hPos); + for (int i = 0; i < len; i++) { byte flag = doReadByte(raw); @@ -2497,10 +2607,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @throws PortableException In case of error. */ private String[] doReadStringArray(boolean raw) throws PortableException { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); String[] arr = new String[len]; + setHandler(arr, hPos); + for (int i = 0; i < len; i++) { byte flag = doReadByte(raw); @@ -2523,10 +2637,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @throws PortableException In case of error. */ private UUID[] doReadUuidArray(boolean raw) throws PortableException { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); UUID[] arr = new UUID[len]; + setHandler(arr, hPos); + for (int i = 0; i < len; i++) { byte flag = doReadByte(raw); @@ -2549,10 +2667,14 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @throws PortableException In case of error. */ private Date[] doReadDateArray(boolean raw) throws PortableException { + int hPos = (raw ? rawOff : off) - 1; + int len = doReadInt(raw); Date[] arr = new Date[len]; + setHandler(arr, hPos); + for (int i = 0; i < len; i++) { byte flag = doReadByte(raw); @@ -2576,12 +2698,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @throws PortableException In case of error. */ private Object[] doReadObjectArray(boolean raw, boolean deep) throws PortableException { + int hPos = (raw ? rawOff : off) - 1; + Class compType = doReadClass(raw); int len = doReadInt(raw); Object[] arr = deep ? (Object[])Array.newInstance(compType, len) : new Object[len]; + setHandler(arr, hPos); + for (int i = 0; i < len; i++) arr[i] = deep ? doReadObject(raw) : unmarshal(raw); @@ -2598,6 +2724,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx @SuppressWarnings("unchecked") private Collection doReadCollection(boolean raw, boolean deep, @Nullable Class cls) throws PortableException { + int hPos = (raw ? rawOff : off) - 1; + int size = doReadInt(raw); assert size >= 0; @@ -2667,6 +2795,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx } } + setHandler(col, hPos); + for (int i = 0; i < size; i++) col.add(deep ? doReadObject(raw) : unmarshal(raw)); @@ -2683,6 +2813,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx @SuppressWarnings("unchecked") private Map doReadMap(boolean raw, boolean deep, @Nullable Class cls) throws PortableException { + int hPos = (raw ? rawOff : off) - 1; + int size = doReadInt(raw); assert size >= 0; @@ -2742,6 +2874,8 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx } } + setHandler(map, hPos); + for (int i = 0; i < size; i++) map.put(deep ? doReadObject(raw) : unmarshal(raw), deep ? doReadObject(raw) : unmarshal(raw)); @@ -2755,10 +2889,16 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx * @throws PortableException In case of error. */ private Map.Entry doReadMapEntry(boolean raw, boolean deep) throws PortableException { + int hPos = (raw ? rawOff : off) - 1; + Object val1 = deep ? doReadObject(raw) : unmarshal(raw); Object val2 = deep ? doReadObject(raw) : unmarshal(raw); - return new GridMapEntry<>(val1, val2); + GridMapEntry entry = new GridMapEntry<>(val1, val2); + + setHandler(entry, hPos); + + return entry; } /** @@ -3017,4 +3157,4 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx @Override public void close() throws IOException { // No-op. } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java index ce77783..7259cc9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java @@ -33,6 +33,7 @@ import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; +import org.apache.ignite.internal.portable.builder.PortableLazyValue; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.portable.PortableObject; import org.jetbrains.annotations.Nullable; @@ -359,6 +360,16 @@ public class PortableUtils { } /** + * Checks whether an array type values can or can not contain references to other object. + * + * @param type Array type. + * @return {@code true} if content of serialized array value cannot contain references to other object. + */ + public static boolean isPlainArrayType(int type) { + return type >= BYTE_ARR && type <= DATE_ARR; + } + + /** * @param cls Class. * @return Portable field type. */ http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java deleted file mode 100644 index ebc68c1..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableValueWithType.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.portable; - -import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl; -import org.apache.ignite.internal.util.typedef.internal.S; - -/** - * - */ -class PortableValueWithType implements PortableLazyValue { - /** */ - private byte type; - - /** */ - private Object val; - - /** - * @param type Type - * @param val Value. - */ - PortableValueWithType(byte type, Object val) { - this.type = type; - this.val = val; - } - - /** {@inheritDoc} */ - @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) { - if (val instanceof PortableBuilderSerializationAware) - ((PortableBuilderSerializationAware)val).writeTo(writer, ctx); - else - ctx.writeValue(writer, val); - } - - /** {@inheritDoc} */ - public String typeName() { - return CacheObjectPortableProcessorImpl.fieldTypeName(type); - } - - /** {@inheritDoc} */ - @Override public Object value() { - if (val instanceof PortableLazyValue) - return ((PortableLazyValue)val).value(); - - return val; - } - - /** - * @param val New value. - */ - public void value(Object val) { - this.val = val; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(PortableValueWithType.class, this); - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java index 6bcce2b..364d5f8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java @@ -156,7 +156,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx * @param off Start offset. * @param typeId Type ID. */ - PortableWriterExImpl(PortableContext ctx, int off, int typeId, boolean metaEnabled) { + public PortableWriterExImpl(PortableContext ctx, int off, int typeId, boolean metaEnabled) { this(ctx, off); this.typeId = typeId; @@ -320,14 +320,14 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx /** * @return Array. */ - byte[] array() { + public byte[] array() { return wCtx.out.arrayCopy(); } /** * @return Output stream. */ - PortableOutputStream outputStream() { + public PortableOutputStream outputStream() { return wCtx.out; } @@ -351,7 +351,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx * @param bytes Number of bytes to reserve. * @return Offset. */ - int reserve(int bytes) { + public int reserve(int bytes) { int pos = wCtx.out.position(); wCtx.out.position(pos + bytes); @@ -363,7 +363,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx * @param bytes Number of bytes to reserve. * @return Offset. */ - int reserveAndMark(int bytes) { + public int reserveAndMark(int bytes) { int off0 = reserve(bytes); mark = wCtx.out.position(); @@ -374,21 +374,21 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx /** * @param off Offset. */ - void writeDelta(int off) { + public void writeDelta(int off) { wCtx.out.writeInt(off, wCtx.out.position() - mark); } /** * */ - void writeLength() { + public void writeLength() { wCtx.out.writeInt(start + TOTAL_LEN_POS, wCtx.out.position() - start); } /** * */ - void writeRawOffsetIfNeeded() { + public void writeRawOffsetIfNeeded() { if (allowFields) wCtx.out.writeInt(start + RAW_DATA_OFF_POS, wCtx.out.position() - start); } @@ -416,63 +416,63 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx /** * @param val Value. */ - void doWriteByte(byte val) { + public void doWriteByte(byte val) { wCtx.out.writeByte(val); } /** * @param val Value. */ - void doWriteShort(short val) { + public void doWriteShort(short val) { wCtx.out.writeShort(val); } /** * @param val Value. */ - void doWriteInt(int val) { + public void doWriteInt(int val) { wCtx.out.writeInt(val); } /** * @param val Value. */ - void doWriteLong(long val) { + public void doWriteLong(long val) { wCtx.out.writeLong(val); } /** * @param val Value. */ - void doWriteFloat(float val) { + public void doWriteFloat(float val) { wCtx.out.writeFloat(val); } /** * @param val Value. */ - void doWriteDouble(double val) { + public void doWriteDouble(double val) { wCtx.out.writeDouble(val); } /** * @param val Value. */ - void doWriteChar(char val) { + public void doWriteChar(char val) { wCtx.out.writeChar(val); } /** * @param val Value. */ - void doWriteBoolean(boolean val) { + public void doWriteBoolean(boolean val) { wCtx.out.writeBoolean(val); } /** * @param val String value. */ - void doWriteDecimal(@Nullable BigDecimal val) { + public void doWriteDecimal(@Nullable BigDecimal val) { if (val == null) doWriteByte(NULL); else { @@ -498,7 +498,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx /** * @param val String value. */ - void doWriteString(@Nullable String val) { + public void doWriteString(@Nullable String val) { if (val == null) doWriteByte(NULL); else { @@ -528,7 +528,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx /** * @param uuid UUID. */ - void doWriteUuid(@Nullable UUID uuid) { + public void doWriteUuid(@Nullable UUID uuid) { if (uuid == null) doWriteByte(NULL); else { @@ -541,7 +541,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx /** * @param date Date. */ - void doWriteDate(@Nullable Date date) { + public void doWriteDate(@Nullable Date date) { if (date == null) doWriteByte(NULL); else { @@ -554,7 +554,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx /** * @param ts Timestamp. */ - void doWriteTimestamp(@Nullable Timestamp ts) { + public void doWriteTimestamp(@Nullable Timestamp ts) { if (ts == null) doWriteByte(NULL); else { @@ -569,7 +569,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx * @param detached Detached or not. * @throws PortableException In case of error. */ - void doWriteObject(@Nullable Object obj, boolean detached) throws PortableException { + public void doWriteObject(@Nullable Object obj, boolean detached) throws PortableException { if (obj == null) doWriteByte(NULL); else { @@ -591,6 +591,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(BYTE_ARR); doWriteInt(val.length); @@ -605,6 +608,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(SHORT_ARR); doWriteInt(val.length); @@ -619,6 +625,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(INT_ARR); doWriteInt(val.length); @@ -633,6 +642,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(LONG_ARR); doWriteInt(val.length); @@ -647,6 +659,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(FLOAT_ARR); doWriteInt(val.length); @@ -661,6 +676,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(DOUBLE_ARR); doWriteInt(val.length); @@ -675,6 +693,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(CHAR_ARR); doWriteInt(val.length); @@ -689,6 +710,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(BOOLEAN_ARR); doWriteInt(val.length); @@ -703,6 +727,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(DECIMAL_ARR); doWriteInt(val.length); @@ -718,6 +745,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(STRING_ARR); doWriteInt(val.length); @@ -733,6 +763,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(UUID_ARR); doWriteInt(val.length); @@ -748,6 +781,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + doWriteByte(DATE_ARR); doWriteInt(val.length); @@ -764,6 +800,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (val == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(val)) + return; + PortableContext.Type type = ctx.typeId(val.getClass().getComponentType()); doWriteByte(OBJ_ARR); @@ -790,6 +829,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (col == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(col)) + return; + doWriteByte(COL); doWriteInt(col.size()); doWriteByte(ctx.collectionType(col.getClass())); @@ -807,6 +849,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (map == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(map)) + return; + doWriteByte(MAP); doWriteInt(map.size()); doWriteByte(ctx.mapType(map.getClass())); @@ -826,6 +871,9 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx if (e == null) doWriteByte(NULL); else { + if (tryWriteAsHandle(e)) + return; + doWriteByte(MAP_ENTRY); doWriteObject(e.getKey(), false); doWriteObject(e.getValue(), false); @@ -905,7 +953,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx /** * @param po Portable object. */ - void doWritePortableObject(@Nullable PortableObjectImpl po) { + public void doWritePortableObject(@Nullable PortableObjectImpl po) { if (po == null) doWriteByte(NULL); else { @@ -1106,64 +1154,88 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx * @param val Value. */ void writeByteArrayField(@Nullable byte[] val) { - doWriteInt(val != null ? 5 + val.length : 1); + int lenPos = reserveAndMark(4); + doWriteByteArray(val); + + writeDelta(lenPos); } /** * @param val Value. */ void writeShortArrayField(@Nullable short[] val) { - doWriteInt(val != null ? 5 + (val.length << 1) : 1); + int lenPos = reserveAndMark(4); + doWriteShortArray(val); + + writeDelta(lenPos); } /** * @param val Value. */ void writeIntArrayField(@Nullable int[] val) { - doWriteInt(val != null ? 5 + (val.length << 2) : 1); + int lenPos = reserveAndMark(4); + doWriteIntArray(val); + + writeDelta(lenPos); } /** * @param val Value. */ void writeLongArrayField(@Nullable long[] val) { - doWriteInt(val != null ? 5 + (val.length << 3) : 1); + int lenPos = reserveAndMark(4); + doWriteLongArray(val); + + writeDelta(lenPos); } /** * @param val Value. */ void writeFloatArrayField(@Nullable float[] val) { - doWriteInt(val != null ? 5 + (val.length << 2) : 1); + int lenPos = reserveAndMark(4); + doWriteFloatArray(val); + + writeDelta(lenPos); } /** * @param val Value. */ void writeDoubleArrayField(@Nullable double[] val) { - doWriteInt(val != null ? 5 + (val.length << 3) : 1); + int lenPos = reserveAndMark(4); + doWriteDoubleArray(val); + + writeDelta(lenPos); } /** * @param val Value. */ void writeCharArrayField(@Nullable char[] val) { - doWriteInt(val != null ? 5 + (val.length << 1) : 1); + int lenPos = reserveAndMark(4); + doWriteCharArray(val); + + writeDelta(lenPos); } /** * @param val Value. */ void writeBooleanArrayField(@Nullable boolean[] val) { - doWriteInt(val != null ? 5 + val.length : 1); + int lenPos = reserveAndMark(4); + doWriteBooleanArray(val); + + writeDelta(lenPos); } /** @@ -1739,12 +1811,31 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx doWriteInt(id); } + /** + * Attempts to write the object as a handle. + * + * @param obj Object to write. + * @return {@code true} if the object has been written as a handle. + */ + boolean tryWriteAsHandle(Object obj) { + int handle = handle(obj); + + if (handle >= 0) { + doWriteByte(GridPortableMarshaller.HANDLE); + doWriteInt(handle); + + return true; + } + + return false; + } + /** * Create new writer with same context. * @param typeId type * @return New writer. */ - PortableWriterExImpl newWriter(int typeId) { + public PortableWriterExImpl newWriter(int typeId) { PortableWriterExImpl res = new PortableWriterExImpl(ctx, wCtx); res.typeId = typeId; @@ -1755,7 +1846,7 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx /** * @return Portable context. */ - PortableContext context() { + public PortableContext context() { return ctx; } @@ -1803,4 +1894,4 @@ public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx handles = new IdentityHashMap<>(); } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java new file mode 100644 index 0000000..1f521ac --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableAbstractLazyValue.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.portable.builder; + +/** + * + */ +abstract class PortableAbstractLazyValue implements PortableLazyValue { + /** */ + protected Object val; + + /** */ + protected final PortableBuilderReader reader; + + /** */ + protected final int valOff; + + /** + * @param reader Reader. + * @param valOff Value. + */ + protected PortableAbstractLazyValue(PortableBuilderReader reader, int valOff) { + this.reader = reader; + this.valOff = valOff; + } + + /** + * @return Value. + */ + protected abstract Object init(); + + /** {@inheritDoc} */ + @Override public Object value() { + if (val == null) { + val = init(); + + assert val != null; + } + + return val; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java new file mode 100644 index 0000000..1472d56 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.portable.builder; + +import org.apache.ignite.internal.portable.GridPortableMarshaller; +import org.apache.ignite.internal.portable.PortableWriterExImpl; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.portable.PortableInvalidClassException; + +/** + * + */ +public class PortableBuilderEnum implements PortableBuilderSerializationAware { + /** */ + private final int ordinal; + + /** */ + private final int typeId; + + /** */ + private final String clsName; + + /** + * @param typeId Type ID. + * @param anEnum Enum instance. + */ + public PortableBuilderEnum(int typeId, Enum anEnum) { + ordinal = anEnum.ordinal(); + this.typeId = typeId; + clsName = null; + } + + /** + * @param reader PortableBuilderReader. + */ + public PortableBuilderEnum(PortableBuilderReader reader) { + int typeId = reader.readInt(); + + if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) { + clsName = reader.readString(); + + Class cls; + + try { + // TODO: IGNITE-1272 - Is class loader needed here? + cls = U.forName(reader.readString(), null); + } + catch (ClassNotFoundException e) { + throw new PortableInvalidClassException("Failed to load the class: " + clsName, e); + } + + this.typeId = reader.portableContext().descriptorForClass(cls).typeId(); + } + else { + this.typeId = typeId; + this.clsName = null; + } + + ordinal = reader.readInt(); + } + + /** + * @return Ordinal. + */ + public int getOrdinal() { + return ordinal; + } + + /** {@inheritDoc} */ + @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) { + writer.writeByte(GridPortableMarshaller.ENUM); + + if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) { + writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID); + writer.writeString(clsName); + } + else + writer.writeInt(typeId); + + writer.writeInt(ordinal); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) + return false; + + PortableBuilderEnum that = (PortableBuilderEnum)o; + + return ordinal == that.ordinal && typeId == that.typeId; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int result = ordinal; + + result = 31 * result + typeId; + + return result; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/9057a4c0/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java new file mode 100644 index 0000000..b2e4c0d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java @@ -0,0 +1,537 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.portable.builder; + +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl; +import org.apache.ignite.internal.util.GridArgumentCheck; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.portable.PortableBuilder; +import org.apache.ignite.portable.PortableException; +import org.apache.ignite.portable.PortableInvalidClassException; +import org.apache.ignite.portable.PortableMetadata; +import org.apache.ignite.portable.PortableObject; +import org.jetbrains.annotations.Nullable; +import org.apache.ignite.internal.portable.*; +import org.apache.ignite.internal.processors.cache.portable.*; +import org.apache.ignite.internal.util.*; +import org.apache.ignite.internal.util.typedef.internal.*; +import org.apache.ignite.portable.*; + +import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS; +import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN; +import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_CODE_POS; +import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS; +import static org.apache.ignite.internal.portable.GridPortableMarshaller.TOTAL_LEN_POS; +import static org.apache.ignite.internal.portable.GridPortableMarshaller.TYPE_ID_POS; +import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID; + +/** + * + */ +public class PortableBuilderImpl implements PortableBuilder { + /** */ + private static final Object REMOVED_FIELD_MARKER = new Object(); + + /** */ + private final PortableContext ctx; + + /** */ + private final int typeId; + + /** May be null. */ + private String typeName; + + /** May be null. */ + private String clsNameToWrite; + + /** */ + private boolean registeredType = true; + + /** */ + private Map assignedVals; + + /** */ + private Map readCache; + + /** Position of object in source array, or -1 if object is not created from PortableObject. */ + private final int start; + + /** Total header length */ + private final int hdrLen; + + /** + * Context of PortableObject reading process. Or {@code null} if object is not created from PortableObject. + */ + private final PortableBuilderReader reader; + + /** */ + private int hashCode; + + /** + * @param clsName Class name. + * @param ctx Portable context. + */ + public PortableBuilderImpl(PortableContext ctx, String clsName) { + this(ctx, ctx.typeId(clsName), PortableContext.typeName(clsName)); + } + + /** + * @param typeId Type ID. + * @param ctx Portable context. + */ + public PortableBuilderImpl(PortableContext ctx, int typeId) { + this(ctx, typeId, null); + } + + /** + * @param typeName Type name. + * @param ctx Context. + * @param typeId Type id. + */ + public PortableBuilderImpl(PortableContext ctx, int typeId, String typeName) { + this.typeId = typeId; + this.typeName = typeName; + this.ctx = ctx; + + start = -1; + reader = null; + hdrLen = DFLT_HDR_LEN; + + readCache = Collections.emptyMap(); + } + + /** + * @param obj Object to wrap. + */ + public PortableBuilderImpl(PortableObjectImpl obj) { + this(new PortableBuilderReader(obj), obj.start()); + + reader.registerObject(this); + } + + /** + * @param reader ctx + * @param start Start. + */ + PortableBuilderImpl(PortableBuilderReader reader, int start) { + this.reader = reader; + this.start = start; + + int typeId = reader.readIntAbsolute(start + TYPE_ID_POS); + ctx = reader.portableContext(); + hashCode = reader.readIntAbsolute(start + HASH_CODE_POS); + + if (typeId == UNREGISTERED_TYPE_ID) { + int mark = reader.position(); + + reader.position(start + CLS_NAME_POS); + + clsNameToWrite = reader.readString(); + + Class cls; + + try { + // TODO: IGNITE-1272 - Is class loader needed here? + cls = U.forName(clsNameToWrite, null); + } + catch (ClassNotFoundException e) { + throw new PortableInvalidClassException("Failed to load the class: " + clsNameToWrite, e); + } + + this.typeId = ctx.descriptorForClass(cls).typeId(); + + registeredType = false; + + hdrLen = reader.position() - mark; + + reader.position(mark); + } + else { + this.typeId = typeId; + hdrLen = DFLT_HDR_LEN; + } + } + + /** {@inheritDoc} */ + @Override public PortableObject build() { + try (PortableWriterExImpl writer = new PortableWriterExImpl(ctx, 0, typeId, false)) { + + PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer(); + + serializationCtx.registerObjectWriting(this, 0); + + serializeTo(writer, serializationCtx); + + byte[] arr = writer.array(); + + return new PortableObjectImpl(ctx, arr, 0); + } + } + + /** + * @param writer Writer. + * @param serializer Serializer. + */ + void serializeTo(PortableWriterExImpl writer, PortableBuilderSerializer serializer) { + writer.doWriteByte(GridPortableMarshaller.OBJ); + writer.doWriteBoolean(true); + writer.doWriteInt(registeredType ? typeId : UNREGISTERED_TYPE_ID); + writer.doWriteInt(hashCode); + + // Length and raw offset. + writer.reserve(8); + + if (!registeredType) + writer.writeString(clsNameToWrite); + + Set remainsFlds = null; + + if (reader != null) { + Map assignedFldsById; + + if (assignedVals != null) { + assignedFldsById = U.newHashMap(assignedVals.size()); + + for (Map.Entry entry : assignedVals.entrySet()) { + int fldId = ctx.fieldId(typeId, entry.getKey()); + + assignedFldsById.put(fldId, entry.getValue()); + } + + remainsFlds = assignedFldsById.keySet(); + } + else + assignedFldsById = Collections.emptyMap(); + + int rawOff = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS); + + reader.position(start + hdrLen); + + int cpStart = -1; + + while (reader.position() < rawOff) { + int fldId = reader.readInt(); + + int len = reader.readInt(); + + if (assignedFldsById.containsKey(fldId)) { + if (cpStart >= 0) { + writer.write(reader.array(), cpStart, reader.position() - 4 - 4 - cpStart); + + cpStart = -1; + } + + Object assignedVal = assignedFldsById.remove(fldId); + + reader.skip(len); + + if (assignedVal != REMOVED_FIELD_MARKER) { + writer.writeInt(fldId); + + int lenPos = writer.reserveAndMark(4); + + serializer.writeValue(writer, assignedVal); + + writer.writeDelta(lenPos); + } + } + else { + int type = len != 0 ? reader.readByte(0) : 0; + + if (len != 0 && !PortableUtils.isPlainArrayType(type) && PortableUtils.isPlainType(type)) { + if (cpStart < 0) + cpStart = reader.position() - 4 - 4; + + reader.skip(len); + } + else { + if (cpStart >= 0) { + writer.write(reader.array(), cpStart, reader.position() - 4 - cpStart); + + cpStart = -1; + } + else + writer.writeInt(fldId); + + Object val; + + if (len == 0) + val = null; + else if (readCache == null) { + int savedPos = reader.position(); + + val = reader.parseValue(); + + assert reader.position() == savedPos + len; + } + else { + val = readCache.get(fldId); + + reader.skip(len); + } + + int lenPos = writer.reserveAndMark(4); + + serializer.writeValue(writer, val); + + writer.writeDelta(lenPos); + } + } + } + + if (cpStart >= 0) + writer.write(reader.array(), cpStart, reader.position() - cpStart); + } + + if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) { + boolean metadataEnabled = ctx.isMetaDataEnabled(typeId); + + PortableMetadata metadata = null; + + if (metadataEnabled) + metadata = ctx.metaData(typeId); + + Map newFldsMetadata = null; + + for (Map.Entry entry : assignedVals.entrySet()) { + Object val = entry.getValue(); + + if (val == REMOVED_FIELD_MARKER) + continue; + + String name = entry.getKey(); + + int fldId = ctx.fieldId(typeId, name); + + if (remainsFlds != null && !remainsFlds.contains(fldId)) + continue; + + writer.writeInt(fldId); + + int lenPos = writer.reserveAndMark(4); + + serializer.writeValue(writer, val); + + writer.writeDelta(lenPos); + + if (metadataEnabled) { + String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name); + + String newFldTypeName; + + if (val instanceof PortableValueWithType) + newFldTypeName = ((PortableValueWithType)val).typeName(); + else { + byte type = PortableUtils.typeByClass(val.getClass()); + + newFldTypeName = CacheObjectPortableProcessorImpl.fieldTypeName(type); + } + + if (oldFldTypeName == null) { + // It's a new field, we have to add it to metadata. + + if (newFldsMetadata == null) + newFldsMetadata = new HashMap<>(); + + newFldsMetadata.put(name, newFldTypeName); + } + else { + if (!"Object".equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) { + throw new PortableException( + "Wrong value has been set [" + + "typeName=" + (typeName == null ? metadata.typeName() : typeName) + + ", fieldName=" + name + + ", fieldType=" + oldFldTypeName + + ", assignedValueType=" + newFldTypeName + + ", assignedValue=" + (((PortableValueWithType)val).value()) + ']' + ); + } + } + } + } + + if (newFldsMetadata != null) { + String typeName = this.typeName; + + if (typeName == null) + typeName = metadata.typeName(); + + ctx.updateMetaData(typeId, typeName, newFldsMetadata); + } + } + + writer.writeRawOffsetIfNeeded(); + + if (reader != null) { + int rawOff = reader.readIntAbsolute(start + RAW_DATA_OFF_POS); + int len = reader.readIntAbsolute(start + TOTAL_LEN_POS); + + if (rawOff < len) + writer.write(reader.array(), rawOff, len - rawOff); + } + + writer.writeLength(); + } + + /** {@inheritDoc} */ + @Override public PortableBuilderImpl hashCode(int hashCode) { + this.hashCode = hashCode; + + return this; + } + + /** + * + */ + private void ensureReadCacheInit() { + if (readCache == null) { + Map readCache = new HashMap<>(); + + int pos = start + hdrLen; + int end = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS); + + while (pos < end) { + int fieldId = reader.readIntAbsolute(pos); + + pos += 4; + + int len = reader.readIntAbsolute(pos); + + pos += 4; + + Object val = reader.getValueQuickly(pos, len); + + readCache.put(fieldId, val); + + pos += len; + } + + this.readCache = readCache; + } + } + + /** {@inheritDoc} */ + @Override public F getField(String name) { + Object val; + + if (assignedVals != null && assignedVals.containsKey(name)) { + val = assignedVals.get(name); + + if (val == REMOVED_FIELD_MARKER) + return null; + } + else { + ensureReadCacheInit(); + + int fldId = ctx.fieldId(typeId, name); + + val = readCache.get(fldId); + } + + return (F)PortableUtils.unwrapLazy(val); + } + + /** {@inheritDoc} */ + @Override public PortableBuilder setField(String name, Object val) { + GridArgumentCheck.notNull(val, name); + + if (assignedVals == null) + assignedVals = new LinkedHashMap<>(); + + Object oldVal = assignedVals.put(name, val); + + if (oldVal instanceof PortableValueWithType) { + ((PortableValueWithType)oldVal).value(val); + + assignedVals.put(name, oldVal); + } + + return this; + } + + /** {@inheritDoc} */ + @Override public PortableBuilder setField(String name, @Nullable T val, Class type) { + if (assignedVals == null) + assignedVals = new LinkedHashMap<>(); + + //int fldId = ctx.fieldId(typeId, fldName); + + assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val)); + + return this; + } + + /** {@inheritDoc} */ + @Override public PortableBuilder setField(String name, @Nullable PortableBuilder builder) { + if (builder == null) + return setField(name, null, Object.class); + else + return setField(name, (Object)builder); + } + + /** + * Removes field from portable object. + * + * @param name Field name. + * @return {@code this} instance for chaining. + */ + @Override public PortableBuilderImpl removeField(String name) { + if (assignedVals == null) + assignedVals = new LinkedHashMap<>(); + + assignedVals.put(name, REMOVED_FIELD_MARKER); + + return this; + } + + /** + * Creates builder initialized by specified portable object. + * + * @param obj Portable object to initialize builder. + * @return New builder. + */ + public static PortableBuilderImpl wrap(PortableObject obj) { + PortableObjectImpl heapObj; + + if (obj instanceof PortableObjectOffheapImpl) + heapObj = (PortableObjectImpl)((PortableObjectOffheapImpl)obj).heapCopy(); + else + heapObj = (PortableObjectImpl)obj; + + return new PortableBuilderImpl(heapObj); + } + + /** + * @return Object start position in source array. + */ + int start() { + return start; + } + + /** + * @return Object type id. + */ + public int typeId() { + return typeId; + } +} \ No newline at end of file