diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/HybridRowHeader.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/HybridRowHeader.java index cb97868..f4d7ff4 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/HybridRowHeader.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/HybridRowHeader.java @@ -10,7 +10,7 @@ public final class HybridRowHeader { /** * Size (in bytes) of a serialized header. */ - public static final int SIZE = (HybridRowVersion.SIZE / Byte.SIZE) + SchemaId.SIZE; + public static final int BYTES = SchemaId.BYTES; private SchemaId schemaId; private HybridRowVersion version = HybridRowVersion.values()[0]; diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/HybridRowVersion.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/HybridRowVersion.java index 56fba38..0284a09 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/HybridRowVersion.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/HybridRowVersion.java @@ -3,7 +3,8 @@ package com.azure.data.cosmos.serialization.hybridrow; -import java.util.HashMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; /** * Versions of HybridRow @@ -12,35 +13,36 @@ import java.util.HashMap; */ public enum HybridRowVersion { - Invalid((byte)0), + INVALID((byte)0), /** * Initial version of the HybridRow format. */ V1((byte)0x81); - public static final int SIZE = java.lang.Byte.SIZE; - private static java.util.HashMap mappings; + public static final int BYTES = Byte.BYTES; + + private static Int2ObjectMap mappings; private byte value; HybridRowVersion(byte value) { this.value = value; - getMappings().put(value, this); + mappings().put(value, this); + } + + public static HybridRowVersion from(byte value) { + return mappings().get(value); } public byte value() { return this.value; } - public static HybridRowVersion from(byte value) { - return getMappings().get(value); - } - - private static java.util.HashMap getMappings() { + private static Int2ObjectMap mappings() { if (mappings == null) { synchronized (HybridRowVersion.class) { if (mappings == null) { - mappings = new HashMap<>(); + mappings = new Int2ObjectOpenHashMap<>(); } } } diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/Result.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/Result.java index 9f03056..f50ce5e 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/Result.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/Result.java @@ -7,43 +7,44 @@ import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; public enum Result { - Success(0), - Failure(1), - NotFound(2), - Exists(3), - TooBig(4), + + SUCCESS(0), + FAILURE(1), + NOT_FOUND(2), + EXISTS(3), + TOO_BIG(4), /** * The type of an existing field does not match the expected type for this operation. */ - TypeMismatch(5), + TYPE_MISMATCH(5), /** * An attempt to write in a read-only scope. */ - InsufficientPermissions(6), + INSUFFICIENT_PERMISSIONS(6), /** * An attempt to write a field that did not match its (optional) type constraints. */ - TypeConstraint(7), + TYPE_CONSTRAINT(7), /** * The byte sequence could not be parsed as a valid row. */ - InvalidRow(8), + INVALID_ROW(8), /** * The byte sequence was too short for the requested action. */ - InsufficientBuffer(9), + INSUFFICIENT_BUFFER(9), /** * The operation was cancelled. */ - Canceled(10); + CANCELED(10); - public static final int SIZE = Integer.SIZE; + public static final int BYTES = Integer.BYTES; private static Int2ObjectMap mappings; private final int value; diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowBuffer.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowBuffer.java index 882e14b..14188fe 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowBuffer.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowBuffer.java @@ -71,6 +71,7 @@ import java.util.Iterator; import java.util.Optional; import java.util.UUID; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; import static com.google.common.base.Preconditions.checkArgument; @@ -129,7 +130,7 @@ public final class RowBuffer { checkNotNull(buffer, "buffer"); checkNotNull(version, "version"); checkNotNull(resolver, "resolver"); - checkArgument(buffer.isReadable(HybridRowHeader.SIZE)); + checkArgument(buffer.isReadable(HybridRowHeader.BYTES)); this.buffer = buffer; this.resolver = resolver; @@ -139,29 +140,71 @@ public final class RowBuffer { Layout layout = resolver.resolve(header.schemaId()); checkState(header.schemaId().equals(layout.schemaId())); - checkState(HybridRowHeader.SIZE + layout.size() <= this.length()); - } - - public Utf8String ReadSparsePath(RowCursor edit) { - - StringTokenizer tokenizer = edit.layout().tokenizer(); - final Optional path = tokenizer.tryFindString(edit.longValue().pathToken); - - if (path.isPresent()) { - return path.get(); - } - - int numBytes = edit.pathToken() - edit.layout().tokenizer().count(); - return Utf8String.fromUnsafe(this.buffer.readSlice(edit.pathOffset(), numBytes)); + checkState(HybridRowHeader.BYTES + layout.size() <= this.length()); } /** - * The length of row in bytes. + * Compute the byte offset from the beginning of the row for a given variable column's value. + * + * @param layout The (optional) layout of the current scope. + * @param scopeOffset The 0-based offset to the beginning of the scope's value. + * @param varIndex The 0-based index of the variable column within the variable segment. + * @return The byte offset from the beginning of the row where the variable column's value should be + * located. */ - //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: - //ORIGINAL LINE: public byte[] ToArray() - public byte[] ToArray() { - return this.buffer.Slice(0, this.length()).ToArray(); + public int ComputeVariableValueOffset(Layout layout, int scopeOffset, int varIndex) { + if (layout == null) { + return scopeOffset; + } + + int index = layout.numFixed() + varIndex; + ReadOnlySpan columns = layout.columns(); + checkState(index <= columns.Length); + int offset = scopeOffset + layout.size(); + for (int i = layout.numFixed(); i < index; i++) { + LayoutColumn col = columns[i]; + if (this.readBit(scopeOffset, col.getNullBit().clone())) { + int lengthSizeInBytes; + Out tempOut_lengthSizeInBytes = new Out(); + //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: + //ORIGINAL LINE: ulong valueSizeInBytes = this.Read7BitEncodedUInt(offset, out int lengthSizeInBytes); + long valueSizeInBytes = this.read7BitEncodedUInt(offset); + lengthSizeInBytes = tempOut_lengthSizeInBytes.get(); + if (col.type().getIsVarint()) { + offset += lengthSizeInBytes; + } else { + offset += (int) valueSizeInBytes + lengthSizeInBytes; + } + } + } + + return offset; + } + + /** + * Delete the sparse field at the indicated path. + * + * @param edit The field to delete. + */ + public void DeleteSparse(RowCursor edit) { + // If the field doesn't exist, then nothing to do. + if (!edit.exists()) { + return; + } + + int numBytes = 0; + int _; + Out tempOut__ = new Out(); + int _; + Out tempOut__2 = new Out(); + int shift; + Out tempOut_shift = new Out(); + this.EnsureSparse(edit, edit.get().cellType(), edit.get().cellTypeArgs().clone(), numBytes, + RowOptions.DELETE, tempOut__, tempOut__2, tempOut_shift); + shift = tempOut_shift.get(); + _ = tempOut__2.get(); + _ = tempOut__.get(); + this.length(this.length() + shift); } public void TypedCollectionMoveField(Reference dstEdit, Reference srcEdit @@ -199,7 +242,7 @@ public final class RowBuffer { int shiftDelete; Out tempOut_shiftDelete = new Out(); this.EnsureSparse(srcEdit, srcEdit.get().cellType(), srcEdit.get().cellTypeArgs().clone(), numBytes, - RowOptions.Delete, tempOut_metaBytes2, tempOut_spaceNeeded2, tempOut_shiftDelete); + RowOptions.DELETE, tempOut_metaBytes2, tempOut_spaceNeeded2, tempOut_shiftDelete); shiftDelete = tempOut_shiftDelete.get(); spaceNeeded = tempOut_spaceNeeded2.get(); metaBytes = tempOut_metaBytes2.get(); @@ -246,7 +289,7 @@ public final class RowBuffer { // converted using the 'Out' helper class unless the method is within the code being modified: scope.get().Clone(out dstEdit); if (dstEdit.count() <= 1) { - return Result.Success; + return Result.SUCCESS; } // Compute Index Elements. @@ -318,7 +361,7 @@ public final class RowBuffer { this.buffer.Slice(this.length(), shift).Fill(0xFF); //#endif - return Result.Success; + return Result.SUCCESS; } public void WriteNullable(Reference edit, LayoutScope scopeType, TypeArgumentList typeArgs, @@ -361,40 +404,33 @@ public final class RowBuffer { this = tempReference_this.get(); } - //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: - //ORIGINAL LINE: internal void WriteSparseBinary(ref RowCursor edit, ReadOnlySpan value, UpdateOptions - // options) - public void WriteSparseBinary(Reference edit, ReadOnlySpan value, UpdateOptions options) { - int len = value.Length; - //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: - //ORIGINAL LINE: int numBytes = len + RowBuffer.Count7BitEncodedUInt((ulong)len); - int numBytes = len + RowBuffer.count7BitEncodedUInt(len); - int metaBytes; - Out tempOut_metaBytes = new Out(); - int spaceNeeded; - Out tempOut_spaceNeeded = new Out(); - int shift; - Out tempOut_shift = new Out(); - this.EnsureSparse(edit, LayoutType.Binary, TypeArgumentList.EMPTY, numBytes, options, tempOut_metaBytes, - tempOut_spaceNeeded, tempOut_shift); - shift = tempOut_shift.get(); - spaceNeeded = tempOut_spaceNeeded.get(); - metaBytes = tempOut_metaBytes.get(); - this.writeSparseMetadata(edit, LayoutType.Binary, TypeArgumentList.EMPTY, metaBytes); - int sizeLenInBytes = this.WriteBinary(edit.get().valueOffset(), value); - checkState(spaceNeeded == metaBytes + len + sizeLenInBytes); - edit.get().endOffset = edit.get().metaOffset() + spaceNeeded; - this.length(this.length() + shift); + public void WriteSparseBinary( + @Nonnull final RowCursor edit, @Nonnull final ByteBuf value, @Nonnull final UpdateOptions options) { + + checkNotNull(edit, "expected non-null edit"); + checkNotNull(value, "expected non-null value"); + checkNotNull(options, "expected non-null options"); + + final LayoutType layoutType = LayoutTypes.BINARY; + final int readableBytes = value.readableBytes(); + final int length = RowBuffer.count7BitEncodedUInt(readableBytes) + readableBytes; + + Out metaBytes = new Out<>(); + Out shift = new Out<>(); + Out spaceNeeded = new Out<>(); + + this.EnsureSparse(edit, layoutType, TypeArgumentList.EMPTY, length, options, metaBytes, spaceNeeded, shift); + this.writeSparseMetadata(edit, layoutType, TypeArgumentList.EMPTY, metaBytes.get()); + this.WriteBinary(edit.valueOffset(), value); + + checkState(spaceNeeded.get() == metaBytes.get() + length); + + edit.endOffset(edit.metaOffset() + spaceNeeded.get()); } - //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: - //ORIGINAL LINE: internal void WriteSparseBinary(ref RowCursor edit, ReadOnlySequence value, UpdateOptions - // options) public void WriteSparseBinary(Reference edit, ReadOnlySequence value, UpdateOptions options) { int len = (int) value.Length; - //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: - //ORIGINAL LINE: int numBytes = len + RowBuffer.Count7BitEncodedUInt((ulong)len); int numBytes = len + RowBuffer.count7BitEncodedUInt(len); int metaBytes; Out tempOut_metaBytes = new Out(); @@ -496,6 +532,28 @@ public final class RowBuffer { this.length(this.length() + shift); } + public void WriteSparseFloat32(@Nonnull RowCursor edit, float value, @Nonnull UpdateOptions options) { + + int numBytes = (Float.SIZE / Byte.SIZE); + int metaBytes; + + Out tempOut_metaBytes = new Out(); + int spaceNeeded; + Out tempOut_spaceNeeded = new Out(); + int shift; + Out tempOut_shift = new Out(); + this.EnsureSparse(edit, LayoutTypes.FLOAT_32, TypeArgumentList.EMPTY, numBytes, options, tempOut_metaBytes, + tempOut_spaceNeeded, tempOut_shift); + shift = tempOut_shift.get(); + spaceNeeded = tempOut_spaceNeeded.get(); + metaBytes = tempOut_metaBytes.get(); + this.writeSparseMetadata(edit, LayoutType.Float32, TypeArgumentList.EMPTY, metaBytes); + this.writeFloat32(edit.get().valueOffset(), value); + checkState(spaceNeeded == metaBytes + (Float.SIZE / Byte.SIZE)); + edit.get().endOffset = edit.get().metaOffset() + spaceNeeded; + this.length(this.length() + shift); + } + public void WriteSparseFloat64(Reference edit, double value, UpdateOptions options) { int numBytes = (Double.SIZE / Byte.SIZE); int metaBytes; @@ -556,6 +614,11 @@ public final class RowBuffer { this.length(this.length() + shift); } + // TODO: DANOBLE: resurrect MongoDbObjectId + // public MongoDbObjectId ReadMongoDbObjectId(int offset) { + // return MemoryMarshal.Read(this.buffer.Slice(offset)); + // } + public void WriteSparseInt32(Reference edit, int value, UpdateOptions options) { int numBytes = (Integer.SIZE / Byte.SIZE); int metaBytes; @@ -576,11 +639,6 @@ public final class RowBuffer { this.length(this.length() + shift); } - // TODO: DANOBLE: resurrect MongoDbObjectId - // public MongoDbObjectId ReadMongoDbObjectId(int offset) { - // return MemoryMarshal.Read(this.buffer.Slice(offset)); - // } - public void WriteSparseInt64(Reference edit, long value, UpdateOptions options) { int numBytes = (Long.SIZE / Byte.SIZE); int metaBytes; @@ -692,6 +750,13 @@ public final class RowBuffer { this.length(this.length() + shift); } + // TODO: DANOBLE: resurrect MongoDbObjectId + // public MongoDbObjectId ReadSparseMongoDbObjectId(Reference edit) { + // this.readSparsePrimitiveTypeCode(edit, MongoDbObjectId); + // edit.get().endOffset = edit.get().valueOffset() + MongoDbObjectId.Size; + // return this.ReadMongoDbObjectId(edit.get().valueOffset()).clone(); + // } + public void WriteSparseString(Reference edit, Utf8Span value, UpdateOptions options) { int len = value.Length; //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -715,13 +780,6 @@ public final class RowBuffer { this.length(this.length() + shift); } - // TODO: DANOBLE: resurrect MongoDbObjectId - // public MongoDbObjectId ReadSparseMongoDbObjectId(Reference edit) { - // this.readSparsePrimitiveTypeCode(edit, MongoDbObjectId); - // edit.get().endOffset = edit.get().valueOffset() + MongoDbObjectId.Size; - // return this.ReadMongoDbObjectId(edit.get().valueOffset()).clone(); - // } - public void WriteSparseTuple(Reference edit, LayoutScope scopeType, TypeArgumentList typeArgs , UpdateOptions options, Out newScope) { int numBytes = (LayoutCode.SIZE / Byte.SIZE) * (1 + typeArgs.count()); // nulls for each element. @@ -1066,42 +1124,6 @@ public final class RowBuffer { this = tempReference_this.get(); } - private void writeUInt16(Short value) { - this.buffer.writeShortLE(value); - } - - private void writeUInt32(Integer value) { - this.buffer.writeIntLE(value); - } - - private void writeUInt64(Long value) { - this.buffer.writeLongLE(value); - } - - private void writeUInt8(Byte value) { - this.buffer.writeByte(value); - } - - public void writeUInt16(int offset, short value) { - Item item = this.write(this::writeUInt16, offset, value); - } - - public void writeUInt32(int offset, int value) { - Item item = this.write(this::writeUInt32, offset, value); - } - - public void writeUInt64(int offset, long value) { - Item item = this.write(this::writeUInt64, offset, value); - } - - public void writeUInt8(int offset, byte value) { - Item item = this.write(this::writeUInt8, offset, value); - } - - public void writeUnixDateTime(int offset, UnixDateTime value) { - Item item = this.write(this::writeUInt64, offset, value.milliseconds()); - } - public void WriteVariableBinary(int offset, ReadOnlySpan value, boolean exists, Out shift) { int numBytes = value.Length; @@ -1171,44 +1193,6 @@ public final class RowBuffer { this.length(this.length() + shift.get()); } - /** - * Compute the byte offset from the beginning of the row for a given variable column's value. - * - * @param layout The (optional) layout of the current scope. - * @param scopeOffset The 0-based offset to the beginning of the scope's value. - * @param varIndex The 0-based index of the variable column within the variable segment. - * @return The byte offset from the beginning of the row where the variable column's value should be - * located. - */ - public int ComputeVariableValueOffset(Layout layout, int scopeOffset, int varIndex) { - if (layout == null) { - return scopeOffset; - } - - int index = layout.numFixed() + varIndex; - ReadOnlySpan columns = layout.columns(); - checkState(index <= columns.Length); - int offset = scopeOffset + layout.size(); - for (int i = layout.numFixed(); i < index; i++) { - LayoutColumn col = columns[i]; - if (this.readBit(scopeOffset, col.getNullBit().clone())) { - int lengthSizeInBytes; - Out tempOut_lengthSizeInBytes = new Out(); - //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: - //ORIGINAL LINE: ulong valueSizeInBytes = this.Read7BitEncodedUInt(offset, out int lengthSizeInBytes); - long valueSizeInBytes = this.read7BitEncodedUInt(offset); - lengthSizeInBytes = tempOut_lengthSizeInBytes.get(); - if (col.type().getIsVarint()) { - offset += lengthSizeInBytes; - } else { - offset += (int) valueSizeInBytes + lengthSizeInBytes; - } - } - } - - return offset; - } - /** * Compute the number of bytes necessary to store the unsigned 32-bit integer value using the varuint encoding * @@ -1231,32 +1215,6 @@ public final class RowBuffer { this.buffer.setIntLE(offset, (int) (value - decrement)); } - /** - * Delete the sparse field at the indicated path. - * - * @param edit The field to delete. - */ - public void DeleteSparse(RowCursor edit) { - // If the field doesn't exist, then nothing to do. - if (!edit.exists()) { - return; - } - - int numBytes = 0; - int _; - Out tempOut__ = new Out(); - int _; - Out tempOut__2 = new Out(); - int shift; - Out tempOut_shift = new Out(); - this.EnsureSparse(edit, edit.get().cellType(), edit.get().cellTypeArgs().clone(), numBytes, - RowOptions.Delete, tempOut__, tempOut__2, tempOut_shift); - shift = tempOut_shift.get(); - _ = tempOut__2.get(); - _ = tempOut__.get(); - this.length(this.length() + shift); - } - public void deleteVariable(int offset, boolean isVarint) { int start = this.buffer.readerIndex(); @@ -1320,8 +1278,8 @@ public final class RowBuffer { @Nonnull public RowCursor prepareSparseMove(@Nonnull final RowCursor scope, @Nonnull final RowCursor srcEdit) { - checkNotNull(srcEdit); - checkNotNull(scope); + checkNotNull(srcEdit, "expected non-null srcEdit"); + checkNotNull(scope, "expected non-null scope"); checkArgument(scope.index() == 0); checkArgument(scope.scopeType().isUniqueScope()); @@ -1400,7 +1358,7 @@ public final class RowBuffer { } public Utf8String readFixedString(int offset, int length) { - Item item = this.read(() -> Utf8String.fromUnsafe(this.buffer.readSlice(length)), offset); + Item item = this.read(this::readFixedString, offset, length); return item.value(); } @@ -1409,14 +1367,6 @@ public final class RowBuffer { return item.value(); } - // TODO: DANOBLE: Support MongoDbObjectId values - // public void WriteMongoDbObjectId(int offset, MongoDbObjectId value) { - // Reference tempReference_value = - // new Reference(value); - // MemoryMarshal.Write(this.buffer.Slice(offset), tempReference_value); - // value = tempReference_value.get(); - // } - public float readFloat32(int offset) { Item item = this.read(this.buffer::readFloatLE, offset); return item.value(); @@ -1445,7 +1395,7 @@ public final class RowBuffer { checkNotNull(inputStream, "expected non-null inputStream"); checkNotNull(resolver, "expected non-null resolver"); checkNotNull(version, "expected non-null version"); - checkState(byteCount >= HybridRowHeader.SIZE, "expected byteCount >= %s, not %s", HybridRowHeader.SIZE, + checkState(byteCount >= HybridRowHeader.BYTES, "expected byteCount >= %s, not %s", HybridRowHeader.BYTES, byteCount); this.reset(); @@ -1481,10 +1431,10 @@ public final class RowBuffer { public boolean readFrom( @Nonnull final ByteBuf input, @Nonnull final HybridRowVersion version, @Nonnull final LayoutResolver resolver) { - checkNotNull(input); - checkNotNull(version); - checkNotNull(resolver); - checkState(input.readableBytes() >= HybridRowHeader.SIZE); + checkNotNull(input, "expected non-null input"); + checkNotNull(version, "expected non-null version"); + checkNotNull(resolver, "expected non-null resolver"); + checkState(input.readableBytes() >= HybridRowHeader.BYTES); this.reset(); this.resolver = resolver; @@ -1525,6 +1475,14 @@ public final class RowBuffer { return item.value(); } + // TODO: DANOBLE: Support MongoDbObjectId values + // public void WriteMongoDbObjectId(int offset, MongoDbObjectId value) { + // Reference tempReference_value = + // new Reference(value); + // MemoryMarshal.Write(this.buffer.Slice(offset), tempReference_value); + // value = tempReference_value.get(); + // } + public ByteBuf readSparseBinary(RowCursor edit) { this.readSparsePrimitiveTypeCode(edit, LayoutTypes.BINARY); Item item = this.read(this::readVariableBinary, edit); @@ -1600,7 +1558,7 @@ public final class RowBuffer { public NullValue readSparseNull(@Nonnull RowCursor edit) { - checkNotNull(edit); + checkNotNull(edit, "expected non-null edit"); this.readSparsePrimitiveTypeCode(edit, LayoutTypes.NULL); edit.endOffset(edit.valueOffset()); @@ -1608,13 +1566,29 @@ public final class RowBuffer { return NullValue.Default; } + public Utf8String readSparsePath(@Nonnull final RowCursor edit) { + + checkNotNull(edit, "expected non-null edit"); + final StringTokenizer tokenizer = edit.layout().tokenizer(); + final Optional path = tokenizer.tryFindString(edit.pathToken()); + + if (path.isPresent()) { + return path.get(); + } + + final int length = edit.pathToken() - tokenizer.count(); + Item item = this.read(this::readFixedString, edit.pathOffset(), length); + + return item.value(); + } + public int readSparsePathLen( @Nonnull final Layout layout, final int offset, @Nonnull final Out pathLenInBytes, @Nonnull final Out pathOffset) { - checkNotNull(layout); - checkNotNull(pathOffset); - checkNotNull(pathLenInBytes); + checkNotNull(layout, "expected non-null layout"); + checkNotNull(pathOffset, "expected non-null pathOffset"); + checkNotNull(pathLenInBytes, "expected non-null pathLenInBytes"); final Item item = this.read(this::read7BitEncodedUInt, offset); final int token = item.value().intValue(); @@ -1958,6 +1932,12 @@ public final class RowBuffer { throw new IllegalStateException(lenientFormat("Not a scope type: %s", scopeType)); } + public byte[] toArray() { + byte[] content = new byte[this.length()]; + this.buffer.getBytes(0, content); + return content; + } + public void unsetBit(final int offset, @Nonnull final LayoutBit bit) { checkNotNull(bit, "expected non-null bit"); checkArgument(!bit.isInvalid()); @@ -1976,12 +1956,11 @@ public final class RowBuffer { * more bytes. An {@link IllegalArgumentException} is thrown, if the specified 64-bit integer value is outside * the range of an unsigned 32-bit integer, [0, 0x00000000FFFFFFFFL]. * - * @param ignored * @param value a 64-bit integer constrained to the range of an unsigned 32-bit integer, [0, 0x00000000FFFFFFFFL] * @return The number of bytes written */ - public int write7BitEncodedUInt(final int ignored, final long value) { - checkArgument(0 <= value && value <= 0x00000000FFFFFFFFL); + public int write7BitEncodedUInt(final long value) { + checkArgument(0 <= value && value <= 0x00000000FFFFFFFFL, "expected value in range [0, %s], not %s", 0x00000000FFFFFFFFL, value); long n = value; int i = 0; while (n >= 0x80L) { @@ -2086,10 +2065,10 @@ public final class RowBuffer { @Nonnull final RowCursor edit, @Nonnull final LayoutScope scopeType, @Nonnull final UpdateOptions options, @Nonnull final Out newScope) { - checkNotNull(edit); - checkNotNull(scopeType); - checkNotNull(options); - checkNotNull(newScope); + checkNotNull(edit, "expected non-null edit"); + checkNotNull(scopeType, "expected non-null scopeType"); + checkNotNull(options, "expected non-null options"); + checkNotNull(newScope, "expected non-null newScope"); int numBytes = (LayoutCode.SIZE / Byte.SIZE); // end scope type code TypeArgumentList typeArgs = TypeArgumentList.EMPTY; @@ -2100,7 +2079,7 @@ public final class RowBuffer { int shift; Out tempOut_shift = new Out(); - this.ensureSparse(edit, scopeType, typeArgs, numBytes, options, tempOut_metaBytes, tempOut_spaceNeeded, + this.EnsureSparse(edit, scopeType, typeArgs, numBytes, options, tempOut_metaBytes, tempOut_spaceNeeded, tempOut_shift); shift = tempOut_shift.get(); spaceNeeded = tempOut_spaceNeeded.get(); @@ -2118,28 +2097,6 @@ public final class RowBuffer { .layout(edit.layout()); } - public void WriteSparseFloat32(@Nonnull RowCursor edit, float value, @Nonnull UpdateOptions options) { - - int numBytes = (Float.SIZE / Byte.SIZE); - int metaBytes; - - Out tempOut_metaBytes = new Out(); - int spaceNeeded; - Out tempOut_spaceNeeded = new Out(); - int shift; - Out tempOut_shift = new Out(); - this.ensureSparse(edit, LayoutTypes.FLOAT_32, TypeArgumentList.EMPTY, numBytes, options, tempOut_metaBytes, - tempOut_spaceNeeded, tempOut_shift); - shift = tempOut_shift.get(); - spaceNeeded = tempOut_spaceNeeded.get(); - metaBytes = tempOut_metaBytes.get(); - this.writeSparseMetadata(edit, LayoutType.Float32, TypeArgumentList.EMPTY, metaBytes); - this.writeFloat32(edit.get().valueOffset(), value); - checkState(spaceNeeded == metaBytes + (Float.SIZE / Byte.SIZE)); - edit.get().endOffset = edit.get().metaOffset() + spaceNeeded; - this.length(this.length() + shift); - } - public void writeSparseTypeCode(int offset, LayoutCode code) { this.writeUInt8(offset, code.value()); } @@ -2155,6 +2112,26 @@ public final class RowBuffer { this.buffer.getBytes(0, stream, this.length()); } + public void writeUInt16(int offset, short value) { + Item item = this.write(this::writeUInt16, offset, value); + } + + public void writeUInt32(int offset, int value) { + Item item = this.write(this::writeUInt32, offset, value); + } + + public void writeUInt64(int offset, long value) { + Item item = this.write(this::writeUInt64, offset, value); + } + + public void writeUInt8(int offset, byte value) { + Item item = this.write(this::writeUInt8, offset, value); + } + + public void writeUnixDateTime(int offset, UnixDateTime value) { + Item item = this.write(this::writeUInt64, offset, value.milliseconds()); + } + /** * Compares the values of two encoded fields using the hybrid row binary collation. * @@ -2270,6 +2247,108 @@ public final class RowBuffer { return RowBuffer.count7BitEncodedUInt(RowBuffer.rotateSignToLsb(value)); } + /** + * Ensure that sufficient space exists in the row buffer to write the specified value + * + * @param edit The prepared edit indicating where and in what context the current write will happen. + * @param cellType The type of the field to be written. + * @param typeArgs The type arguments of the field to be written. + * @param numBytes The number of bytes needed to encode the value of the field to be written. + * @param options The kind of edit to be performed. + * @param metaBytes On success, the number of bytes needed to encode the metadata of the new field. + * @param spaceNeeded On success, the number of bytes needed in total to encode the new field and its metadata. + * @param shift On success, the number of bytes the length of the row buffer was increased. + */ + private void EnsureSparse( + @Nonnull final RowCursor edit, @Nonnull final LayoutType cellType, @Nonnull final TypeArgumentList typeArgs, + final int numBytes, @Nonnull final RowOptions options, @Nonnull final Out metaBytes, + @Nonnull final Out spaceNeeded, @Nonnull final Out shift) { + + int metaOffset = edit.metaOffset(); + int spaceAvailable = 0; + + // Compute the metadata offsets + + if (edit.scopeType().hasImplicitTypeCode(edit)) { + metaBytes.setAndGet(0); + } else { + metaBytes.setAndGet(cellType.countTypeArgument(typeArgs)); + } + + if (!edit.scopeType().isIndexedScope()) { + checkState(edit.writePath() != null); + int pathLenInBytes = RowBuffer.countSparsePath(edit); + metaBytes.setAndGet(metaBytes.get() + pathLenInBytes); + } + + if (edit.exists()) { + // Compute value offset for existing value to be overwritten. + spaceAvailable = this.sparseComputeSize(edit); + } + + spaceNeeded.setAndGet(options == RowOptions.DELETE ? 0 : metaBytes.get() + numBytes); + shift.setAndGet(spaceNeeded.get() - spaceAvailable); + if (shift.get() > 0) { + this.ensure(this.length() + shift.get()); + } + + this.buffer.Slice(metaOffset + spaceAvailable, this.length() - (metaOffset + spaceAvailable)).CopyTo(this.buffer.Slice(metaOffset + spaceNeeded.get())); + + // TODO: C# TO JAVA CONVERTER: There is no preprocessor in Java: + //#if DEBUG + if (shift.get() < 0) { + // Fill deleted bits (in debug builds) to detect overflow/alignment errors. + this.buffer.Slice(this.length() + shift.get(), -shift.get()).Fill(0xFF); + } + //#endif + + // Update the stored size (fixed arity scopes don't store the size because it is implied by the type args). + if (edit.scopeType().isSizedScope() && !edit.scopeType().isFixedArity()) { + if ((options == RowOptions.INSERT) || (options == RowOptions.INSERT_AT) || ((options == RowOptions.UPSERT) && !edit.get().exists())) { + // Add one to the current scope count. + checkState(!edit.exists()); + this.incrementUInt32(edit.start(), 1); + edit.count(edit.count() + 1); + } else if ((options == RowOptions.DELETE) && edit.exists()) { + // Subtract one from the current scope count. + checkState(this.readUInt32(edit.start()) > 0); + this.decrementUInt32(edit.start(), 1); + edit.count(edit.count() - 1); + } + } + + if (options == RowOptions.DELETE) { + edit.cellType(null); + edit.cellTypeArgs(null); + edit.exists(false); + } else { + edit.cellType(cellType); + edit.cellTypeArgs(typeArgs); + edit.exists(true); + } + } + + /** + * Ensure that sufficient space exists in the row buffer to write the specified value + * + * @param edit The prepared edit indicating where and in what context the current write will happen. + * @param cellType The type of the field to be written. + * @param typeArgs The type arguments of the field to be written. + * @param numBytes The number of bytes needed to encode the value of the field to be written. + * @param options The kind of edit to be performed. + * @param metaBytes On success, the number of bytes needed to encode the metadata of the new field. + * @param spaceNeeded On success, the number of bytes needed in total to encode the new field and its metadata. + * @param shift On success, the number of bytes the length of the row buffer was increased. + */ + private void EnsureSparse( + @Nonnull final RowCursor edit, @Nonnull final LayoutType cellType, @Nonnull final TypeArgumentList typeArgs, + final int numBytes, @Nonnull final UpdateOptions options, @Nonnull final Out metaBytes, + @Nonnull final Out spaceNeeded, @Nonnull final Out shift) { + checkNotNull(options, "expected non-null options"); + RowOptions rowOptions = RowOptions.from(options.value()); + this.EnsureSparse(edit, cellType, typeArgs, numBytes, rowOptions, metaBytes, spaceNeeded, shift); + } + private void EnsureVariable(int offset, boolean isVarint, int numBytes, boolean exists, Out spaceNeeded, Out shift) { int spaceAvailable = 0; @@ -2374,15 +2453,17 @@ public final class RowBuffer { return true; } - private int WriteBinary(int offset, ReadOnlySpan value) { - int sizeLenInBytes = this.write7BitEncodedUInt(offset, (long) value.Length); - value.CopyTo(this.buffer.Slice(offset + sizeLenInBytes)); - return sizeLenInBytes; + private void writeVariableBinary(ByteBuf value) { + this.write7BitEncodedUInt(value.readableBytes()); + this.buffer.writeBytes(value); + } + + private int WriteBinary(int offset, ByteBuf value) { + Item item = this.write(this::writeVariableBinary, offset, value); + return item.length(); } private int WriteBinary(int offset, ReadOnlySequence value) { - //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: - //ORIGINAL LINE: int sizeLenInBytes = this.Write7BitEncodedUInt(offset, (ulong)value.Length); int sizeLenInBytes = this.write7BitEncodedUInt(offset, (long) value.Length); value.CopyTo(this.buffer.Slice(offset + sizeLenInBytes)); return sizeLenInBytes; @@ -2516,138 +2597,40 @@ public final class RowBuffer { this.buffer.ensureWritable(size); } - /** - * Ensure that sufficient space exists in the row buffer to write the current value. - * - * @param edit The prepared edit indicating where and in what context the current write will - * happen. - * @param cellType The type of the field to be written. - * @param typeArgs The type arguments of the field to be written. - * @param numBytes The number of bytes needed to encode the value of the field to be written. - * @param options The kind of edit to be performed. - * @param metaBytes On success, the number of bytes needed to encode the metadata of the new - * field. - * @param spaceNeeded On success, the number of bytes needed in total to encode the new field - * and its metadata. - * @param shift On success, the number of bytes the length of the row buffer was increased - */ - private void ensureSparse( - @Nonnull final RowCursor edit, @Nonnull final LayoutType cellType, @Nonnull final TypeArgumentList typeArgs, - final int numBytes, @Nonnull final RowOptions options, @Nonnull final Out metaBytes, - @Nonnull final Out spaceNeeded, @Nonnull final Out shift) { + private Item read(@Nonnull final Supplier reader, @Nonnull final RowCursor cursor) { - int metaOffset = edit.metaOffset(); - int spaceAvailable = 0; - - // Compute the metadata offsets - if (edit.scopeType().hasImplicitTypeCode(edit)) { - metaBytes.setAndGet(0); - } else { - metaBytes.setAndGet(cellType.countTypeArgument(typeArgs)); - } - - if (!edit.scopeType().isIndexedScope()) { - checkState(edit.writePath() != null); - int pathLenInBytes = RowBuffer.countSparsePath(edit); - metaBytes.setAndGet(metaBytes.get() + pathLenInBytes); - } - - if (edit.exists()) { - // Compute value offset for existing value to be overwritten. - spaceAvailable = this.sparseComputeSize(edit); - } - - spaceNeeded.setAndGet(options == RowOptions.Delete ? 0 : metaBytes.get() + numBytes); - shift.setAndGet(spaceNeeded.get() - spaceAvailable); - if (shift.get() > 0) { - this.ensure(this.length() + shift.get()); - } - - this.buffer.Slice(metaOffset + spaceAvailable, this.length() - (metaOffset + spaceAvailable)).CopyTo(this.buffer.Slice(metaOffset + spaceNeeded.get())); - - // TODO: C# TO JAVA CONVERTER: There is no preprocessor in Java: - //#if DEBUG - if (shift.get() < 0) { - // Fill deleted bits (in debug builds) to detect overflow/alignment errors. - this.buffer.Slice(this.length() + shift.get(), -shift.get()).Fill(0xFF); - } - //#endif - - // Update the stored size (fixed arity scopes don't store the size because it is implied by the type args). - if (edit.scopeType().isSizedScope() && !edit.scopeType().isFixedArity()) { - if ((options == RowOptions.Insert) || (options == RowOptions.InsertAt) || ((options == RowOptions.Upsert) && !edit.get().exists())) { - // Add one to the current scope count. - checkState(!edit.exists()); - this.incrementUInt32(edit.start(), 1); - edit.count(edit.count() + 1); - } else if ((options == RowOptions.Delete) && edit.exists()) { - // Subtract one from the current scope count. - checkState(this.readUInt32(edit.start()) > 0); - this.decrementUInt32(edit.start(), 1); - edit.count(edit.count() - 1); - } - } - - if (options == RowOptions.Delete) { - edit.cellType(null); - edit.cellTypeArgs(null); - edit.exists(false); - } else { - edit.cellType(cellType); - edit.cellTypeArgs(typeArgs); - edit.exists(true); - } - } - - /** - * - * . - */ - private void ensureSparse( - @Nonnull final RowCursor edit, @Nonnull final LayoutType cellType, @Nonnull final TypeArgumentList typeArgs, - final int numBytes, @Nonnull final UpdateOptions options, @Nonnull final Out metaBytes, - @Nonnull final Out spaceNeeded, @Nonnull final Out shift) { - - checkNotNull(edit); - checkNotNull(cellType); - checkNotNull(typeArgs); - checkNotNull(options); - checkNotNull(metaBytes); - checkNotNull(spaceNeeded); - checkNotNull(shift); - - final RowOptions rowOptions = RowOptions.from(options.value()); - this.ensureSparse(edit, cellType, typeArgs, numBytes, rowOptions, metaBytes, spaceNeeded, shift); - } - - private Item read(@Nonnull final Supplier supplier, @Nonnull RowCursor cursor) { - checkNotNull(supplier, "expected non-null supplier"); + checkNotNull(reader, "expected non-null supplier"); checkNotNull(cursor, "expected non-null cursor"); - Item item = this.read(supplier, cursor.valueOffset()); + + Item item = this.read(reader, cursor.valueOffset()); cursor.endOffset(this.buffer.readerIndex()); + return item; } - private Item read(@Nonnull final Supplier supplier, int offset) { + private Item read(@Nonnull final Supplier reader, int offset) { - checkNotNull(supplier); + checkNotNull(reader, "expected non-null reader"); + checkArgument(offset >= 0, "expected non-negative offset, not %s", offset); this.buffer.readerIndex(offset); - T value = supplier.get(); + final T value = reader.get(); return Item.of(value, offset, this.buffer.readerIndex() - offset); } - private T read(@Nonnull final Supplier supplier, int offset, @Nonnull Out lengthInBytes) { + private Item read(@Nonnull final Function reader, final int offset, final int length) { - checkNotNull(supplier); - checkNotNull(lengthInBytes); + checkNotNull(reader, "expected non-null reader"); + checkArgument(offset >= 0, "expected non-negative offset, not %s", offset); + checkArgument(length >= 0, "expected non-negative length, not %s", length); - Item item = this.read(supplier, offset); - lengthInBytes.set(item.length()); + this.buffer.readerIndex(offset); + final T value = reader.apply(length); + final int actualLength = this.buffer.readerIndex() - offset; + checkState(actualLength == length, "expected read of length %s, not %s", length, actualLength); - return item.value(); + return Item.of(value, offset, actualLength); } private long read7BitEncodedInt() { @@ -2679,6 +2662,10 @@ public final class RowBuffer { return DecimalCodec.decode(this.buffer); } + private Utf8String readFixedString(int length) { + return Utf8String.fromUnsafe(this.buffer.readSlice(length)); + } + private Float128 readFloat128() { return Float128Codec.decode(this.buffer); } @@ -2721,7 +2708,7 @@ public final class RowBuffer { */ private void readSparseMetadata(@Nonnull final RowCursor edit) { - checkNotNull(edit); + checkNotNull(edit, "expected non-null edit"); if (edit.scopeType().hasImplicitTypeCode(edit)) { edit.scopeType().setImplicitTypeCode(edit); @@ -2945,15 +2932,14 @@ public final class RowBuffer { */ private boolean validateHeader(@Nonnull final HybridRowVersion version) { - checkNotNull(version); + checkNotNull(version, "expected non-null version"); - Out lengthInBytes = new Out<>(); - HybridRowHeader header = this.read(this::readHeader, 0, lengthInBytes); + final Item item = this.read(this::readHeader, 0); + final HybridRowHeader header = item.value(); + final Layout layout = this.resolver.resolve(header.schemaId()); - Layout layout = this.resolver.resolve(header.schemaId()); checkState(header.schemaId().equals(layout.schemaId())); - - return header.version().equals(version) && (HybridRowHeader.SIZE + layout.size()) <= this.length(); + return header.version().equals(version) && (HybridRowHeader.BYTES + layout.size()) <= this.length(); } private Item write(@Nonnull final Consumer consumer, final int offset, @Nonnull final T value) { @@ -3142,6 +3128,7 @@ public final class RowBuffer { this.writeSparsePath(edit, metaOffset); edit.valueOffset(edit.metaOffset() + metaBytes); + checkState(edit.valueOffset() == edit.metaOffset() + metaBytes); } @@ -3184,6 +3171,22 @@ public final class RowBuffer { return sizeLenInBytes; } + private void writeUInt16(Short value) { + this.buffer.writeShortLE(value); + } + + private void writeUInt32(Integer value) { + this.buffer.writeIntLE(value); + } + + private void writeUInt64(Long value) { + this.buffer.writeLongLE(value); + } + + private void writeUInt8(Byte value) { + this.buffer.writeByte(value); + } + private static class Item { private int length; diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowCursor.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowCursor.java index 6091bf7..1cb4894 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowCursor.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowCursor.java @@ -52,13 +52,13 @@ public final class RowCursor implements Cloneable { final SchemaId schemaId = row.readSchemaId(1); final Layout layout = row.resolver().resolve(schemaId); - final int sparseSegmentOffset = row.ComputeVariableValueOffset(layout, HybridRowHeader.SIZE, layout.numVariable()); + final int sparseSegmentOffset = row.ComputeVariableValueOffset(layout, HybridRowHeader.BYTES, layout.numVariable()); return new RowCursor() .layout(layout) .scopeType(LayoutTypes.UDT) .scopeTypeArgs(new TypeArgumentList(schemaId)) - .start(HybridRowHeader.SIZE) + .start(HybridRowHeader.BYTES) .metaOffset(sparseSegmentOffset) .valueOffset(sparseSegmentOffset); } @@ -72,7 +72,7 @@ public final class RowCursor implements Cloneable { .layout(layout) .scopeType(LayoutTypes.UDT) .scopeTypeArgs(new TypeArgumentList(schemaId)) - .start(HybridRowHeader.SIZE) + .start(HybridRowHeader.BYTES) .metaOffset(row.length()) .valueOffset(row.length()); } diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowCursors.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowCursors.java index b4ed49e..64aa1f7 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowCursors.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowCursors.java @@ -23,7 +23,7 @@ public final class RowCursors { if (!(edit.cellType() instanceof LayoutEndScope)) { while (row.sparseIteratorMoveNext(edit)) { - if (path.equals(row.ReadSparsePath(edit))) { + if (path.equals(row.readSparsePath(edit))) { edit.exists(true); break; } diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowOptions.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowOptions.java index 7648e70..dc18a70 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowOptions.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/RowOptions.java @@ -3,13 +3,15 @@ package com.azure.data.cosmos.serialization.hybridrow; -import java.util.HashMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; /** * Describes the desired behavior when mutating a hybrid row. */ public enum RowOptions { - None(0), + + NONE(0), /** * Overwrite an existing value. @@ -18,7 +20,7 @@ public enum RowOptions { * replaced inline. The remainder of the row is resized to accomodate either an increase or decrease * in required space. */ - Update(1), + UPDATE(1), /** * Insert a new value. @@ -27,23 +29,23 @@ public enum RowOptions { * inserted immediately at the offset. The remainder of the row is resized to accomodate either an * increase or decrease in required space. */ - Insert(2), + INSERT(2), /** * Update an existing value or insert a new value, if no value exists. *

- * If a value exists, then this operation becomes {@link Update}, otherwise it - * becomes {@link Insert}. + * If a value exists, then this operation becomes {@link #UPDATE}, otherwise it + * becomes {@link #INSERT}. */ - Upsert(3), + UPSERT(3), /** * Insert a new value moving existing values to the right. *

* Within an array scope, inserts a new value immediately at the index moving all subsequent - * items to the right. In any other scope behaves the same as {@link Upsert}. + * items to the right. In any other scope behaves the same as {@link #UPSERT}. */ - InsertAt(4), + INSERT_AT(4), /** * Delete an existing value. @@ -51,30 +53,31 @@ public enum RowOptions { * If a value exists, then it is removed. The remainder of the row is resized to accomodate * a decrease in required space. If no value exists this operation is a no-op. */ - Delete(5); + DELETE(5); - public static final int SIZE = java.lang.Integer.SIZE; - private static HashMap mappings; - private int value; + public static final int BYTES = Integer.BYTES; + + private static Int2ObjectMap mappings; + private final int value; RowOptions(int value) { this.value = value; mappings().put(value, this); } - public int getValue() { - return this.value; - } - public static RowOptions from(int value) { return mappings().get(value); } - private static HashMap mappings() { + public int value() { + return this.value; + } + + private static Int2ObjectMap mappings() { if (mappings == null) { synchronized (RowOptions.class) { if (mappings == null) { - mappings = new HashMap<>(); + mappings = new Int2ObjectOpenHashMap<>(); } } } diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/SchemaId.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/SchemaId.java index 964e50d..0fe9305 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/SchemaId.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/SchemaId.java @@ -28,9 +28,9 @@ import static com.google.common.base.Strings.lenientFormat; @JsonSerialize(using = SchemaId.JsonSerializer.class) public final class SchemaId { + public static final int BYTES = Integer.BYTES; public static final SchemaId INVALID = null; public static final SchemaId NONE = new SchemaId(-1); - public static final int SIZE = Integer.SIZE / Byte.SIZE; private static final long MAX_VALUE = 0x00000000FFFFFFFFL; private static final Int2ReferenceMap cache = new Int2ReferenceOpenHashMap<>(); @@ -104,7 +104,7 @@ public final class SchemaId { final long value = parser.getLongValue(); if (value < 0 || value > MAX_VALUE) { - String message = lenientFormat("expected integer value in [0, 4294967295], not %s", value); + String message = lenientFormat("expected value in [0, 4294967295], not %s", value); throw MismatchedInputException.from(parser, SchemaId.class, message); } diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowReader.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowReader.java index 1f4ecac..3b579f2 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowReader.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowReader.java @@ -143,7 +143,7 @@ public final class RowReader { this.cursor = cursor.get(); Reference row = new Reference<>(this.row); Reference tempReference_nullableScope = new Reference<>(nullableScope); - boolean tempVar = LayoutNullable.hasValue(row, tempReference_nullableScope) == Result.Success; + boolean tempVar = LayoutNullable.hasValue(row, tempReference_nullableScope) == Result.SUCCESS; nullableScope = tempReference_nullableScope.get(); this.row = row.get(); return tempVar; @@ -196,7 +196,7 @@ public final class RowReader { Reference tempReference_cursor = new Reference(this.cursor); - Utf8Span span = this.row.ReadSparsePath(tempReference_cursor); + Utf8Span span = this.row.readSparsePath(tempReference_cursor); this.cursor = tempReference_cursor.get(); return Utf8String.CopyFrom(span); default: @@ -216,7 +216,7 @@ public final class RowReader { case Sparse: Reference tempReference_cursor = new Reference(this.cursor); - Utf8Span tempVar = this.row.ReadSparsePath(tempReference_cursor); + Utf8Span tempVar = this.row.readSparsePath(tempReference_cursor); this.cursor = tempReference_cursor.get(); return tempVar; default: @@ -347,7 +347,7 @@ public final class RowReader { //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: Result r = this.ReadBinary(out ReadOnlySpan span); Result r = this.ReadBinary(out span); - value.setAndGet((r == Result.Success) ? span.ToArray() :) + value.setAndGet((r == Result.SUCCESS) ? span.ToArray() :) default return r; } @@ -367,17 +367,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutBinary)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseBinary(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } @@ -394,17 +394,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutBoolean)) { value.setAndGet(false); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseBoolean(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(false); - return Result.Failure; + return Result.FAILURE; } } @@ -421,17 +421,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutDateTime)) { value.setAndGet(LocalDateTime.MIN); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseDateTime(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(LocalDateTime.MIN); - return Result.Failure; + return Result.FAILURE; } } @@ -448,17 +448,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutDecimal)) { value.setAndGet(new BigDecimal(0)); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseDecimal(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(new BigDecimal(0)); - return Result.Failure; + return Result.FAILURE; } } @@ -475,17 +475,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutFloat128)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseFloat128(tempReference_cursor).clone()); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } @@ -502,17 +502,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutFloat32)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseFloat32(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -529,17 +529,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutFloat64)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseFloat64(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -556,17 +556,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutGuid)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseGuid(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } @@ -583,17 +583,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutInt16)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseInt16(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -610,17 +610,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutInt32)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseInt32(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -637,17 +637,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutInt64)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseInt64(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -664,17 +664,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutInt8)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseInt8(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -691,17 +691,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutMongoDbObjectId)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.ReadSparseMongoDbObjectId(tempReference_cursor).clone()); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } @@ -718,17 +718,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutNull)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseNull(tempReference_cursor).clone()); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } @@ -772,9 +772,9 @@ public final class RowReader { Reference tempReference_nestedReader = new Reference(nestedReader); // TODO: C# TO JAVA CONVERTER: The following line could not be converted: - Result result = func == null ? null : func.Invoke(ref nestedReader, context) ??Result.Success; + Result result = func == null ? null : func.Invoke(ref nestedReader, context) ??Result.SUCCESS; nestedReader = tempReference_nestedReader.get(); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } @@ -786,7 +786,7 @@ public final class RowReader { tempReference_cursor2); nestedReader.cursor = tempReference_cursor2.get(); this.row = tempReference_row2.get(); - return Result.Success; + return Result.SUCCESS; } /** @@ -800,7 +800,7 @@ public final class RowReader { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = this.ReadString(out span); - value.setAndGet((r == Result.Success) ? span.toString() :) + value.setAndGet((r == Result.SUCCESS) ? span.toString() :) default return r; } @@ -816,7 +816,7 @@ public final class RowReader { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = this.ReadString(out span); - value.setAndGet((r == Result.Success) ? Utf8String.CopyFrom(span) :) + value.setAndGet((r == Result.SUCCESS) ? Utf8String.CopyFrom(span) :) default return r; } @@ -834,17 +834,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutUtf8)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseString(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } @@ -863,17 +863,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutUInt16)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseUInt16(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -892,17 +892,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutUInt32)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseUInt32(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -921,17 +921,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutUInt64)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseUInt64(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -950,17 +950,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutUInt8)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseUInt8(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -977,17 +977,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutUnixDateTime)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseUnixDateTime(tempReference_cursor).clone()); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } @@ -1004,17 +1004,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutVarInt)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseVarInt(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -1033,17 +1033,17 @@ public final class RowReader { case Sparse: if (!(this.cursor.cellType() instanceof LayoutVarUInt)) { value.setAndGet(0); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } Reference tempReference_cursor = new Reference(this.cursor); value.setAndGet(this.row.readSparseVarUInt(tempReference_cursor)); this.cursor = tempReference_cursor.get(); - return Result.Success; + return Result.SUCCESS; default: value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } } @@ -1062,7 +1062,7 @@ public final class RowReader { */ public Result SkipScope(Reference nestedReader) { if (nestedReader.get().cursor.start() != this.cursor.valueOffset()) { - return Result.Failure; + return Result.FAILURE; } Reference tempReference_row = @@ -1073,7 +1073,7 @@ public final class RowReader { tempReference_cursor); nestedReader.get().argValue.cursor = tempReference_cursor.get(); this.row = tempReference_row.get(); - return Result.Success; + return Result.SUCCESS; } public RowReader clone() { @@ -1101,7 +1101,7 @@ public final class RowReader { LayoutType t = this.columns[this.columnIndex].Type; if (!(t instanceof LayoutType)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } switch (col == null ? null : col.storage()) { @@ -1124,7 +1124,7 @@ public final class RowReader { default: throw new IllegalStateException(); value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } @@ -1139,7 +1139,7 @@ public final class RowReader { LayoutType t = this.columns[this.columnIndex].Type; if (!(t instanceof ILayoutUtf8SpanReadable)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } switch (col == null ? null : col.storage()) { @@ -1161,7 +1161,7 @@ public final class RowReader { default: throw new IllegalStateException(); value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } @@ -1177,7 +1177,7 @@ public final class RowReader { LayoutType t = this.columns[this.columnIndex].Type; if (!(t instanceof ILayoutSpanReadable)) { value.setAndGet(null); - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } switch (col == null ? null : col.storage()) { @@ -1198,7 +1198,7 @@ public final class RowReader { default: throw new IllegalStateException(); value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowReaderExtensions.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowReaderExtensions.java index e5f4479..8715536 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowReaderExtensions.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowReaderExtensions.java @@ -42,29 +42,29 @@ public final class RowReaderExtensions { Result r3 = ctx2.Deserializer.invoke(tempReference_itemReader, tempOut_op); op = tempOut_op.get(); itemReader = tempReference_itemReader.get(); - if (r3 != Result.Success) { + if (r3 != Result.SUCCESS) { return r3; } ctx2.List.add(op); - return Result.Success; + return Result.SUCCESS; }); - if (r2 != Result.Success) { + if (r2 != Result.SUCCESS) { return r2; } } - return Result.Success; + return Result.SUCCESS; }); - if (r != Result.Success) { + if (r != Result.SUCCESS) { list.setAndGet(null); return r; } list.setAndGet(ctx.List); - return Result.Success; + return Result.SUCCESS; } /** diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowWriter.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowWriter.java index 94ed203..f5b2a90 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowWriter.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/io/RowWriter.java @@ -372,7 +372,7 @@ public final class RowWriter { WriterFunc func) { LayoutType type = typeArg.type(); Result result = this.PrepareSparseWrite(path, typeArg.clone()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } @@ -530,7 +530,7 @@ public final class RowWriter { break; default: - return Result.Failure; + return Result.FAILURE; } Reference tempReference_row = @@ -543,12 +543,12 @@ public final class RowWriter { Reference tempReference_nestedWriter = new Reference(nestedWriter); // TODO: C# TO JAVA CONVERTER: The following line could not be converted: - result = func == null ? null : func.Invoke(ref nestedWriter, typeArg, context) ??Result.Success; + result = func == null ? null : func.Invoke(ref nestedWriter, typeArg, context) ??Result.SUCCESS; nestedWriter = tempReference_nestedWriter.get(); this.row = nestedWriter.row.clone(); nestedScope.count(nestedWriter.cursor.count()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { // TODO: what about unique violations here? return result; } @@ -558,7 +558,7 @@ public final class RowWriter { new Reference(nestedScope); result = this.row.TypedCollectionUniqueIndexRebuild(tempReference_nestedScope2); nestedScope = tempReference_nestedScope2.get(); - if (result != Result.Success) { + if (result != Result.SUCCESS) { // TODO: If the index rebuild fails then the row is corrupted. Should we automatically clean up here? return result; } @@ -572,7 +572,7 @@ public final class RowWriter { , tempReference_cursor12); nestedWriter.cursor = tempReference_cursor12.get(); this.row = tempReference_row2.get(); - return Result.Success; + return Result.SUCCESS; } /** @@ -765,23 +765,23 @@ public final class RowWriter { private Result PrepareSparseWrite(UtfAnyString path, TypeArgument typeArg) { if (this.cursor.scopeType().isFixedArity() && !(this.cursor.scopeType() instanceof LayoutNullable)) { if ((this.cursor.index() < this.cursor.scopeTypeArgs().count()) && !typeArg.equals(this.cursor.scopeTypeArgs().get(this.cursor.index()).clone())) { - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } } else if (this.cursor.scopeType() instanceof LayoutTypedMap) { Reference tempReference_cursor = new Reference(this.cursor); if (!typeArg.equals(this.cursor.scopeType().typeAs().FieldType(tempReference_cursor).clone())) { this.cursor = tempReference_cursor.get(); - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } else { this.cursor = tempReference_cursor.get(); } } else if (this.cursor.scopeType().isTypedScope() && !typeArg.equals(this.cursor.scopeTypeArgs().get(0).clone())) { - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } this.cursor.writePath(path); - return Result.Success; + return Result.SUCCESS; } /** @@ -795,15 +795,15 @@ public final class RowWriter { * @return Success if the write is successful, an error code otherwise. */ private & ILayoutUtf8SpanWritable> Result WritePrimitive(UtfAnyString path, Utf8Span value, TLayoutType type, AccessUtf8SpanMethod sparse) { - Result result = Result.NotFound; + Result result = Result.NOT_FOUND; if (this.cursor.scopeType() instanceof LayoutUDT) { result = this.WriteSchematizedValue(path, value); } - if (result == Result.NotFound) { + if (result == Result.NOT_FOUND) { // Write sparse value. result = this.PrepareSparseWrite(path, type.getTypeArg().clone()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } @@ -834,15 +834,15 @@ public final class RowWriter { * @return Success if the write is successful, an error code otherwise. */ private & ILayoutSpanWritable, TElement> Result WritePrimitive(UtfAnyString path, ReadOnlySpan value, TLayoutType type, AccessReadOnlySpanMethod sparse) { - Result result = Result.NotFound; + Result result = Result.NOT_FOUND; if (this.cursor.scopeType() instanceof LayoutUDT) { result = this.WriteSchematizedValue(path, value); } - if (result == Result.NotFound) { + if (result == Result.NOT_FOUND) { // Write sparse value. result = this.PrepareSparseWrite(path, type.getTypeArg().clone()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } @@ -873,15 +873,15 @@ public final class RowWriter { * @return Success if the write is successful, an error code otherwise. */ private & ILayoutSequenceWritable, TElement> Result WritePrimitive(UtfAnyString path, ReadOnlySequence value, TLayoutType type, AccessMethod> sparse) { - Result result = Result.NotFound; + Result result = Result.NOT_FOUND; if (this.cursor.scopeType() instanceof LayoutUDT) { result = this.WriteSchematizedValue(path, value); } - if (result == Result.NotFound) { + if (result == Result.NOT_FOUND) { // Write sparse value. result = this.PrepareSparseWrite(path, type.getTypeArg().clone()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } @@ -912,16 +912,16 @@ public final class RowWriter { */ private Result WritePrimitive(UtfAnyString path, TValue value, LayoutType type, AccessMethod sparse) { - Result result = Result.NotFound; + Result result = Result.NOT_FOUND; if (this.cursor.scopeType() instanceof LayoutUDT) { result = this.WriteSchematizedValue(path, value); } - if (result == Result.NotFound) { + if (result == Result.NOT_FOUND) { // Write sparse value. result = this.PrepareSparseWrite(path, type.getTypeArg().clone()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } @@ -953,13 +953,13 @@ public final class RowWriter { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: if (!this.cursor.layout().TryFind(path, out col)) { - return Result.NotFound; + return Result.NOT_FOUND; } boolean tempVar = col.Type instanceof LayoutType; LayoutType t = tempVar ? (LayoutType)col.Type : null; if (!(tempVar)) { - return Result.NotFound; + return Result.NOT_FOUND; } switch (col.Storage) { @@ -978,10 +978,10 @@ public final class RowWriter { return tempVar3; default: - return Result.NotFound; + return Result.NOT_FOUND; } - return Result.NotFound; + return Result.NOT_FOUND; } /** @@ -996,12 +996,12 @@ public final class RowWriter { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: if (!this.cursor.layout().TryFind(path, out col)) { - return Result.NotFound; + return Result.NOT_FOUND; } LayoutType t = col.Type; if (!(t instanceof ILayoutUtf8SpanWritable)) { - return Result.NotFound; + return Result.NOT_FOUND; } switch (col.Storage) { @@ -1027,7 +1027,7 @@ public final class RowWriter { this.row = tempReference_row2.get(); return tempVar2; default: - return Result.NotFound; + return Result.NOT_FOUND; } } @@ -1043,12 +1043,12 @@ public final class RowWriter { LayoutColumn col; // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these cannot be converted using the 'Out' helper class unless the method is within the code being modified: if (!this.cursor.layout().TryFind(path, out col)) { - return Result.NotFound; + return Result.NOT_FOUND; } LayoutType t = col.Type; if (!(t instanceof ILayoutSpanWritable)) { - return Result.NotFound; + return Result.NOT_FOUND; } switch (col.Storage) { @@ -1067,7 +1067,7 @@ public final class RowWriter { this.row = tempReference_row2.get(); return tempVar2; default: - return Result.NotFound; + return Result.NOT_FOUND; } } @@ -1083,12 +1083,12 @@ public final class RowWriter { LayoutColumn col; // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these cannot be converted using the 'Out' helper class unless the method is within the code being modified: if (!this.cursor.layout().TryFind(path, out col)) { - return Result.NotFound; + return Result.NOT_FOUND; } LayoutType t = col.Type; if (!(t instanceof ILayoutSequenceWritable)) { - return Result.NotFound; + return Result.NOT_FOUND; } switch (col.Storage) { @@ -1107,7 +1107,7 @@ public final class RowWriter { this.row = tempReference_row2.get(); return tempVar2; default: - return Result.NotFound; + return Result.NOT_FOUND; } } diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/json/RowReaderJsonExtensions.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/json/RowReaderJsonExtensions.java index 8cabafd..768ad7a 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/json/RowReaderJsonExtensions.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/json/RowReaderJsonExtensions.java @@ -40,7 +40,7 @@ public final class RowReaderJsonExtensions { ctx.Builder.append("{"); Result result = RowReaderJsonExtensions.ToJson(reader, ctx.clone()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { str.setAndGet(null); return result; } @@ -48,7 +48,7 @@ public final class RowReaderJsonExtensions { ctx.Builder.append(ctx.NewLine); ctx.Builder.append("}"); str.setAndGet(ctx.Builder.toString()); - return Result.Success; + return Result.SUCCESS; } private static Result ToJson(Reference reader, ReaderStringContext ctx) { @@ -78,7 +78,7 @@ public final class RowReaderJsonExtensions { new Out(); r = reader.get().ReadNull(tempOut__); _ = tempOut__.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -91,7 +91,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value = new Out(); r = reader.get().ReadBool(tempOut_value); value = tempOut_value.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -104,7 +104,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value2 = new Out(); r = reader.get().ReadInt8(tempOut_value2); value = tempOut_value2.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -117,7 +117,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value3 = new Out(); r = reader.get().ReadInt16(tempOut_value3); value = tempOut_value3.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -130,7 +130,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value4 = new Out(); r = reader.get().ReadInt32(tempOut_value4); value = tempOut_value4.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -143,7 +143,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value5 = new Out(); r = reader.get().ReadInt64(tempOut_value5); value = tempOut_value5.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -158,7 +158,7 @@ public final class RowReaderJsonExtensions { //ORIGINAL LINE: r = reader.ReadUInt8(out byte value); r = reader.get().ReadUInt8(tempOut_value6); value = tempOut_value6.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -173,7 +173,7 @@ public final class RowReaderJsonExtensions { //ORIGINAL LINE: r = reader.ReadUInt16(out ushort value); r = reader.get().ReadUInt16(tempOut_value7); value = tempOut_value7.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -188,7 +188,7 @@ public final class RowReaderJsonExtensions { //ORIGINAL LINE: r = reader.ReadUInt32(out uint value); r = reader.get().ReadUInt32(tempOut_value8); value = tempOut_value8.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -203,7 +203,7 @@ public final class RowReaderJsonExtensions { //ORIGINAL LINE: r = reader.ReadUInt64(out ulong value); r = reader.get().ReadUInt64(tempOut_value9); value = tempOut_value9.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -216,7 +216,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value10 = new Out(); r = reader.get().ReadVarInt(tempOut_value10); value = tempOut_value10.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -231,7 +231,7 @@ public final class RowReaderJsonExtensions { //ORIGINAL LINE: r = reader.ReadVarUInt(out ulong value); r = reader.get().ReadVarUInt(tempOut_value11); value = tempOut_value11.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -244,7 +244,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value12 = new Out(); r = reader.get().ReadFloat32(tempOut_value12); value = tempOut_value12.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -257,7 +257,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value13 = new Out(); r = reader.get().ReadFloat64(tempOut_value13); value = tempOut_value13.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -271,7 +271,7 @@ public final class RowReaderJsonExtensions { new Out(); r = reader.get().ReadFloat128(tempOut__2); _ = tempOut__2.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -285,7 +285,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value14 = new Out(); r = reader.get().ReadDecimal(tempOut_value14); value = tempOut_value14.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -298,7 +298,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value15 = new Out(); r = reader.get().ReadDateTime(tempOut_value15); value = tempOut_value15.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -314,7 +314,7 @@ public final class RowReaderJsonExtensions { new Out(); r = reader.get().ReadUnixDateTime(tempOut_value16); value = tempOut_value16.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -327,7 +327,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value17 = new Out(); r = reader.get().ReadGuid(tempOut_value17); value = tempOut_value17.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -342,7 +342,7 @@ public final class RowReaderJsonExtensions { Out tempOut_value18 = new Out(); r = reader.get().ReadMongoDbObjectId(tempOut_value18); value = tempOut_value18.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -361,7 +361,7 @@ public final class RowReaderJsonExtensions { // - these cannot be converted using the 'Out' helper class unless the method is within the // code being modified: r = reader.get().ReadString(out value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -377,7 +377,7 @@ public final class RowReaderJsonExtensions { //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: r = reader.ReadBinary(out ReadOnlySpan value); r = reader.get().ReadBinary(out value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -429,7 +429,7 @@ public final class RowReaderJsonExtensions { ctx.Builder.append(scopeBracket); int snapshot = ctx.Builder.length(); r = reader.get().ReadScope(new ReaderStringContext(ctx.Builder, ctx.Settings.clone(), ctx.Indent + 1), RowReaderJsonExtensions.ToJson); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -449,7 +449,7 @@ public final class RowReaderJsonExtensions { } } - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Java does not allow user-defined value types. The behavior of this class may differ from the original: diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutArray.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutArray.java index 0a0e5f9..5eb95e4 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutArray.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutArray.java @@ -40,12 +40,12 @@ public final class LayoutArray extends LayoutIndexedScope { public Result writeScope(RowBuffer b, RowCursor edit, TypeArgumentList typeArgs, UpdateOptions options, Out value) { Result result = prepareSparseWrite(b, edit, this.typeArg(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.writeSparseArray(edit, this, options, value); - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutBinary.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutBinary.java index 419b165..814165d 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutBinary.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutBinary.java @@ -40,7 +40,7 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: Result r = this.ReadFixed(ref b, ref scope, col, out ReadOnlySpan span); Result r = this.ReadFixed(b, scope, column, out span); - value.setAndGet((r == Result.Success) ? span.ToArray() :) + value.setAndGet((r == Result.SUCCESS) ? span.ToArray() :) default return r; } @@ -54,11 +54,11 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa checkArgument(col.getSize() >= 0); if (!b.get().readBit(scope.get().start(), col.getNullBit().clone())) { value.setAndGet(null); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().readFixedBinary(scope.get().start() + col.getOffset(), col.getSize())); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -71,7 +71,7 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: Result r = this.ReadSparse(ref b, ref edit, out ReadOnlySpan span); Result r = this.ReadSparse(b, edit, out span); - value.setAndGet((r == Result.Success) ? span.ToArray() :) + value.setAndGet((r == Result.SUCCESS) ? span.ToArray() :) default return r; } @@ -80,13 +80,13 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa //ORIGINAL LINE: public Result ReadSparse(ref RowBuffer b, ref RowCursor edit, out ReadOnlySpan value) public Result ReadSparse(Reference b, Reference edit, Out> value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } value.setAndGet(b.get().readSparseBinary(edit)); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -101,7 +101,7 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: Result r = this.ReadVariable(ref b, ref scope, col, out ReadOnlySpan span); Result r = this.ReadVariable(b, scope, column, out span); - value.setAndGet((r == Result.Success) ? span.ToArray() :) + value.setAndGet((r == Result.SUCCESS) ? span.ToArray() :) default return r; } @@ -114,13 +114,13 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), col.getNullBit().clone())) { value.setAndGet(null); - return Result.NotFound; + return Result.NOT_FOUND; } int varOffset = b.get().ComputeVariableValueOffset(scope.get().layout(), scope.get().start(), col.getOffset()); value.setAndGet(b.get().readVariableBinary(varOffset)); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -144,12 +144,12 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa checkArgument(col.getSize() >= 0); checkArgument(value.Length == col.getSize()); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().WriteFixedBinary(scope.get().start() + col.getOffset(), value, col.getSize()); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -161,12 +161,12 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa checkArgument(col.getSize() >= 0); checkArgument(value.Length == col.getSize()); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().WriteFixedBinary(scope.get().start() + col.getOffset(), value, col.getSize()); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } @Override @@ -199,12 +199,12 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa public Result WriteSparse(Reference b, Reference edit, ReadOnlySpan value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseBinary(edit, value, options); - return Result.Success; + return Result.SUCCESS; } public Result WriteSparse(Reference b, Reference edit, @@ -219,12 +219,12 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa public Result WriteSparse(Reference b, Reference edit, ReadOnlySequence value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseBinary(edit, value, options); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -246,12 +246,12 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa LayoutColumn col, ReadOnlySpan value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } int length = value.Length; if ((col.getSize() > 0) && (length > col.getSize())) { - return Result.TooBig; + return Result.TOO_BIG; } boolean exists = b.get().readBit(scope.get().start(), col.getNullBit().clone()); @@ -264,7 +264,7 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa b.get().setBit(scope.get().start(), col.getNullBit().clone()); scope.get().metaOffset(scope.get().metaOffset() + shift); scope.get().valueOffset(scope.get().valueOffset() + shift); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -274,12 +274,12 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa LayoutColumn col, ReadOnlySequence value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } int length = (int)value.Length; if ((col.getSize() > 0) && (length > col.getSize())) { - return Result.TooBig; + return Result.TOO_BIG; } boolean exists = b.get().readBit(scope.get().start(), col.getNullBit().clone()); @@ -292,6 +292,6 @@ public final class LayoutBinary extends LayoutType implements ILayoutSpa b.get().setBit(scope.get().start(), col.getNullBit().clone()); scope.get().metaOffset(scope.get().metaOffset() + shift); scope.get().valueOffset(scope.get().valueOffset() + shift); - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutBoolean.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutBoolean.java index 9ec2573..3a16eac 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutBoolean.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutBoolean.java @@ -34,24 +34,24 @@ public final class LayoutBoolean extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(false); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().readBit(scope.get().start(), column.getBooleanBit().clone())); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(false); return result; } value.setAndGet(b.get().ReadSparseBool(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -59,7 +59,7 @@ public final class LayoutBoolean extends LayoutType { boolean value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } if (value) { @@ -69,7 +69,7 @@ public final class LayoutBoolean extends LayoutType { } b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -79,12 +79,12 @@ public final class LayoutBoolean extends LayoutType { public Result WriteSparse(Reference b, Reference edit, boolean value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseBool(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutDateTime.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutDateTime.java index eed664a..3d9826a 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutDateTime.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutDateTime.java @@ -32,24 +32,24 @@ public final class LayoutDateTime extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), col.getNullBit().clone())) { value.setAndGet(LocalDateTime.MIN); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().readDateTime(scope.get().start() + col.getOffset())); - return Result.Success; + return Result.SUCCESS; } @Override public Result ReadSparse(Reference b, Reference edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(LocalDateTime.MIN); return result; } value.setAndGet(b.get().readSparseDateTime(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -57,12 +57,12 @@ public final class LayoutDateTime extends LayoutType { LocalDateTime value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeDateTime(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -71,12 +71,12 @@ public final class LayoutDateTime extends LayoutType { public Result writeSparse(Reference b, Reference edit, LocalDateTime value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseDateTime(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutDecimal.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutDecimal.java index aa5dc81..21fa640 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutDecimal.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutDecimal.java @@ -32,24 +32,24 @@ public final class LayoutDecimal extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(new BigDecimal(0)); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadDecimal(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(new BigDecimal(0)); return result; } value.setAndGet(b.get().ReadSparseDecimal(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -57,12 +57,12 @@ public final class LayoutDecimal extends LayoutType { BigDecimal value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().WriteDecimal(scope.get().start() + column.getOffset(), value); b.get().SetBit(scope.get().start(), column.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -72,12 +72,12 @@ public final class LayoutDecimal extends LayoutType { public Result writeSparse(RowBuffer b, RowCursor edit, BigDecimal value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseDecimal(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutEndScope.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutEndScope.java index 62a8e7a..ecec16d 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutEndScope.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutEndScope.java @@ -37,6 +37,6 @@ public final class LayoutEndScope extends LayoutScope { TypeArgumentList typeArgs, UpdateOptions options, Out value) { Contract.Fail("Cannot write an EndScope directly"); value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } } \ No newline at end of file diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutFloat128.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutFloat128.java index ceae32e..d5b1d24 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutFloat128.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutFloat128.java @@ -30,24 +30,24 @@ public final class LayoutFloat128 extends LayoutType value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } value.setAndGet(b.get().ReadSparseFloat128(edit).clone()); - return Result.Success; + return Result.SUCCESS; } @Override @@ -55,12 +55,12 @@ public final class LayoutFloat128 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadFloat32(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseFloat32(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -55,12 +55,12 @@ public final class LayoutFloat32 extends LayoutType { float value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeFloat32(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -70,12 +70,12 @@ public final class LayoutFloat32 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, float value, UpdateOptions options) { Result result = prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseFloat32(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutFloat64.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutFloat64.java index f3734d1..c7baa49 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutFloat64.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutFloat64.java @@ -30,24 +30,24 @@ public final class LayoutFloat64 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadFloat64(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseFloat64(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -55,12 +55,12 @@ public final class LayoutFloat64 extends LayoutType { double value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeFloat64(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -70,12 +70,12 @@ public final class LayoutFloat64 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, double value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseFloat64(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutGuid.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutGuid.java index e0c272a..d1a38b5 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutGuid.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutGuid.java @@ -31,24 +31,24 @@ public final class LayoutGuid extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(null); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadGuid(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } value.setAndGet(b.get().ReadSparseGuid(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -56,12 +56,12 @@ public final class LayoutGuid extends LayoutType { UUID value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().WriteGuid(scope.get().start() + column.getOffset(), value); b.get().SetBit(scope.get().start(), column.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -71,12 +71,12 @@ public final class LayoutGuid extends LayoutType { public Result writeSparse(RowBuffer b, RowCursor edit, UUID value, UpdateOptions options) { Result result = prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseGuid(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt16.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt16.java index dafe1a9..a22309f 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt16.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt16.java @@ -30,24 +30,24 @@ public final class LayoutInt16 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadInt16(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseInt16(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -55,12 +55,12 @@ public final class LayoutInt16 extends LayoutType { short value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeInt16(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -70,12 +70,12 @@ public final class LayoutInt16 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, short value, UpdateOptions options) { Result result = prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseInt16(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt32.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt32.java index f0dff7d..7392412 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt32.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt32.java @@ -30,24 +30,24 @@ public final class LayoutInt32 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadInt32(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseInt32(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -55,12 +55,12 @@ public final class LayoutInt32 extends LayoutType { int value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeInt32(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -70,12 +70,12 @@ public final class LayoutInt32 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, int value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseInt32(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt64.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt64.java index 511d05a..90d3301 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt64.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt64.java @@ -30,24 +30,24 @@ public final class LayoutInt64 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadInt64(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseInt64(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -55,12 +55,12 @@ public final class LayoutInt64 extends LayoutType { long value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeInt64(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -70,12 +70,12 @@ public final class LayoutInt64 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, long value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseInt64(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt8.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt8.java index 66a5cb1..1ea59cf 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt8.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutInt8.java @@ -30,24 +30,24 @@ public final class LayoutInt8 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadInt8(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseInt8(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -55,12 +55,12 @@ public final class LayoutInt8 extends LayoutType { byte value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeInt8(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -70,12 +70,12 @@ public final class LayoutInt8 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, byte value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseInt8(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutMongoDbObjectId.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutMongoDbObjectId.java index c066e01..ccca46b 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutMongoDbObjectId.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutMongoDbObjectId.java @@ -30,24 +30,24 @@ public final class LayoutMongoDbObjectId extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(null); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadMongoDbObjectId(scope.get().start() + column.getOffset()).clone()); - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } value.setAndGet(b.get().ReadSparseMongoDbObjectId(edit).clone()); - return Result.Success; + return Result.SUCCESS; } @Override @@ -55,12 +55,12 @@ public final class LayoutMongoDbObjectId extends LayoutType { MongoDbObjectId value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().WriteMongoDbObjectId(scope.get().start() + column.getOffset(), value.clone()); b.get().SetBit(scope.get().start(), column.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -70,12 +70,12 @@ public final class LayoutMongoDbObjectId extends LayoutType { public Result writeSparse(RowBuffer b, RowCursor edit, MongoDbObjectId value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseMongoDbObjectId(edit, value.clone(), options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutNull.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutNull.java index ebb5421..29c656d 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutNull.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutNull.java @@ -34,23 +34,23 @@ public final class LayoutNull extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); value.setAndGet(NullValue.Default); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { - return Result.NotFound; + return Result.NOT_FOUND; } - return Result.Success; + return Result.SUCCESS; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } value.setAndGet(b.get().readSparseNull(edit).clone()); - return Result.Success; + return Result.SUCCESS; } @Override @@ -58,11 +58,11 @@ public final class LayoutNull extends LayoutType { NullValue value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().SetBit(scope.get().start(), column.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -72,12 +72,12 @@ public final class LayoutNull extends LayoutType { public Result writeSparse(RowBuffer b, RowCursor edit, NullValue value, UpdateOptions options) { Result result = prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseNull(edit, value.clone(), options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutNullable.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutNullable.java index 63521a0..792c025 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutNullable.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutNullable.java @@ -47,7 +47,7 @@ public final class LayoutNullable extends LayoutIndexedScope { checkArgument(scope.index() == 1 || scope.index() == 2); checkArgument(scope.scopeTypeArgs().count() == 1); boolean hasValue = b.readInt8(scope.start()) != 0; - return hasValue ? Result.Success : Result.NotFound; + return hasValue ? Result.SUCCESS : Result.NOT_FOUND; } @Override @@ -75,13 +75,13 @@ public final class LayoutNullable extends LayoutIndexedScope { TypeArgumentList typeArgs, boolean hasValue, Out value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, new TypeArgument(this, typeArgs.clone()), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteNullable(edit, this, typeArgs.clone(), options, hasValue, value.clone()); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutObject.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutObject.java index cd0ec6d..4e9b2e4 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutObject.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutObject.java @@ -39,12 +39,12 @@ public final class LayoutObject extends LayoutPropertyScope { public Result writeScope(RowBuffer b, RowCursor edit, TypeArgumentList typeArgs, UpdateOptions options, Out value) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteSparseObject(edit, this, options, value.clone()); - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutScope.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutScope.java index ab295f5..6fcba65 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutScope.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutScope.java @@ -75,12 +75,12 @@ public abstract class LayoutScope extends LayoutType { Result result = LayoutType.prepareSparseDelete(b, edit, this.layoutCode()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.DeleteSparse(edit); - return Result.Success; + return Result.SUCCESS; } /** @@ -105,14 +105,14 @@ public abstract class LayoutScope extends LayoutType { Result result = LayoutType.prepareSparseRead(b, edit, this.layoutCode()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } boolean immutable = this.isImmutable() || edit.immutable() || edit.scopeType().isUniqueScope(); value.set(b.sparseIteratorReadScope(edit, immutable)); - return Result.Success; + return Result.SUCCESS; } public void readSparsePath(@Nonnull final RowBuffer row, @Nonnull final RowCursor edit) { @@ -159,7 +159,7 @@ public abstract class LayoutScope extends LayoutType { final Out out = new Out<>(); Result result = this.writeScope(b, scope, typeArgs, options, out); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } @@ -167,14 +167,14 @@ public abstract class LayoutScope extends LayoutType { if (func != null) { result = func.invoke(b, childScope, context); - if (result != Result.Success) { + if (result != Result.SUCCESS) { this.deleteScope(b, scope); return result; } } RowCursors.skip(scope, b, childScope); - return Result.Success; + return Result.SUCCESS; } /** diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTagged.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTagged.java index e482d72..e8a3d33 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTagged.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTagged.java @@ -61,13 +61,13 @@ public final class LayoutTagged extends LayoutIndexedScope { public Result writeScope(RowBuffer b, RowCursor edit, TypeArgumentList typeArgs, UpdateOptions options, Out value) { Result result = prepareSparseWrite(b, edit, new TypeArgument(this, typeArgs.clone()), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteTypedTuple(edit, this, typeArgs.clone(), options, value.clone()); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTagged2.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTagged2.java index edcbc39..9dc898d 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTagged2.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTagged2.java @@ -74,13 +74,13 @@ public final class LayoutTagged2 extends LayoutIndexedScope { public Result writeScope(RowBuffer b, RowCursor edit, TypeArgumentList typeArgs, UpdateOptions options, Out value) { Result result = prepareSparseWrite(b, edit, new TypeArgument(this, typeArgs.clone()), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteTypedTuple(edit, this, typeArgs.clone(), options, value.clone()); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTuple.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTuple.java index bec9fe0..a5ecf64 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTuple.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTuple.java @@ -62,13 +62,13 @@ public final class LayoutTuple extends LayoutIndexedScope { public Result writeScope(RowBuffer b, RowCursor edit, TypeArgumentList typeArgs, UpdateOptions options, Out value) { Result result = prepareSparseWrite(b, edit, new TypeArgument(this, typeArgs.clone()), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteSparseTuple(edit, this, typeArgs.clone(), options, value.clone()); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutType.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutType.java index 87f5973..4eb8ce6 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutType.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutType.java @@ -99,16 +99,16 @@ public abstract class LayoutType implements ILayoutType { checkArgument(scope.scopeType() instanceof LayoutUDT); if (scope.immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } if (column.nullBit().isInvalid()) { // Cannot delete a non-nullable fixed column. - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } b.unsetBit(scope.start(), column.nullBit()); - return Result.Success; + return Result.SUCCESS; } /** @@ -123,12 +123,12 @@ public abstract class LayoutType implements ILayoutType { Result result = LayoutType.prepareSparseDelete(b, edit, this.layoutCode()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.DeleteSparse(edit); - return Result.Success; + return Result.SUCCESS; } /** @@ -142,7 +142,7 @@ public abstract class LayoutType implements ILayoutType { checkArgument(scope.scopeType() instanceof LayoutUDT); if (scope.immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } boolean exists = b.readBit(scope.start(), column.nullBit()); @@ -153,7 +153,7 @@ public abstract class LayoutType implements ILayoutType { b.unsetBit(scope.start(), column.nullBit()); } - return Result.Success; + return Result.SUCCESS; } public static LayoutType fromCode(LayoutCode code) { @@ -164,9 +164,9 @@ public abstract class LayoutType implements ILayoutType { public final Result hasValue(RowBuffer b, RowCursor scope, LayoutColumn column) { if (!b.readBit(scope.start(), column.nullBit())) { - return Result.NotFound; + return Result.NOT_FOUND; } - return Result.Success; + return Result.SUCCESS; } /** @@ -192,18 +192,18 @@ public abstract class LayoutType implements ILayoutType { public static Result prepareSparseDelete(RowBuffer b, RowCursor edit, LayoutCode code) { if (edit.scopeType().isFixedArity()) { - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } if (edit.immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } if (edit.exists() && LayoutCodeTraits.Canonicalize(edit.cellType().layoutCode()) != code) { - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } - return Result.Success; + return Result.SUCCESS; } /** @@ -234,32 +234,32 @@ public abstract class LayoutType implements ILayoutType { // Prepare the delete of the source Result result = LayoutType.prepareSparseDelete(b, srcEdit, elementType.type().layoutCode()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { dstEdit.setAndGet(null); return result; } if (!srcEdit.exists()) { dstEdit.setAndGet(null); - return Result.NotFound; + return Result.NOT_FOUND; } if (destinationScope.immutable()) { b.DeleteSparse(srcEdit); dstEdit.setAndGet(null); - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } if (!srcEdit.cellTypeArgs().equals(elementType.typeArgs())) { b.DeleteSparse(srcEdit); dstEdit.setAndGet(null); - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } if (options == UpdateOptions.InsertAt) { b.DeleteSparse(srcEdit); dstEdit.setAndGet(null); - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } // Prepare the insertion at the destination. @@ -267,16 +267,16 @@ public abstract class LayoutType implements ILayoutType { if ((options == UpdateOptions.Update) && (!dstEdit.get().exists())) { b.DeleteSparse(srcEdit); dstEdit.setAndGet(null); - return Result.NotFound; + return Result.NOT_FOUND; } if ((options == UpdateOptions.Insert) && dstEdit.get().exists()) { b.DeleteSparse(srcEdit); dstEdit.setAndGet(null); - return Result.Exists; + return Result.EXISTS; } - return Result.Success; + return Result.SUCCESS; } /** @@ -291,14 +291,14 @@ public abstract class LayoutType implements ILayoutType { @Nonnull final RowBuffer b, @Nonnull final RowCursor edit, @Nonnull LayoutCode code) { if (!edit.exists()) { - return Result.NotFound; + return Result.NOT_FOUND; } if (LayoutCodeTraits.Canonicalize(edit.cellType().layoutCode()) != code) { - return Result.TypeMismatch; + return Result.TYPE_MISMATCH; } - return Result.Success; + return Result.SUCCESS; } /** @@ -315,23 +315,23 @@ public abstract class LayoutType implements ILayoutType { @Nonnull final UpdateOptions options) { if (edit.immutable() || (edit.scopeType().isUniqueScope() && !edit.deferUniqueIndex())) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } if (edit.scopeType().isFixedArity() && !(edit.scopeType() instanceof LayoutNullable)) { if ((edit.index() < edit.scopeTypeArgs().count()) && !typeArg.equals(edit.scopeTypeArgs().get(edit.index()))) { - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } } else if (edit.scopeType() instanceof LayoutTypedMap) { if (!((typeArg.type() instanceof LayoutTypedTuple) && typeArg.typeArgs().equals(edit.scopeTypeArgs()))) { - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } } else if (edit.scopeType().isTypedScope() && !typeArg.equals(edit.scopeTypeArgs().get(0))) { - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } if ((options == UpdateOptions.InsertAt) && edit.scopeType().isFixedArity()) { - return Result.TypeConstraint; + return Result.TYPE_CONSTRAINT; } if ((options == UpdateOptions.InsertAt) && !edit.scopeType().isFixedArity()) { @@ -339,14 +339,14 @@ public abstract class LayoutType implements ILayoutType { } if ((options == UpdateOptions.Update) && (!edit.exists())) { - return Result.NotFound; + return Result.NOT_FOUND; } if ((options == UpdateOptions.Insert) && edit.exists()) { - return Result.Exists; + return Result.EXISTS; } - return Result.Success; + return Result.SUCCESS; } public abstract Result readFixed(RowBuffer b, RowCursor scope, LayoutColumn column, Out value); @@ -370,7 +370,7 @@ public abstract class LayoutType implements ILayoutType { public Result readVariable(RowBuffer b, RowCursor scope, LayoutColumn column, Out value) { value.setAndGet(null); - return Result.Failure; + return Result.FAILURE; } /** @@ -400,7 +400,7 @@ public abstract class LayoutType implements ILayoutType { } public Result writeVariable(Reference b, Reference scope, LayoutColumn col, T value) { - return Result.Failure; + return Result.FAILURE; } TypeArgument typeArg() { diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedArray.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedArray.java index 850be2b..8d7a8f8 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedArray.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedArray.java @@ -56,13 +56,13 @@ public final class LayoutTypedArray extends LayoutIndexedScope { public Result writeScope(RowBuffer b, RowCursor edit, TypeArgumentList typeArgs, UpdateOptions options, Out value) { Result result = LayoutType.prepareSparseWrite(b, edit, new TypeArgument(this, typeArgs.clone()), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteTypedArray(edit, this, typeArgs.clone(), options, value.clone()); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedMap.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedMap.java index 480e467..776a59d 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedMap.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedMap.java @@ -77,13 +77,13 @@ public final class LayoutTypedMap extends LayoutUniqueScope { public Result writeScope(RowBuffer b, RowCursor edit, TypeArgumentList typeArgs, UpdateOptions options, Out value) { Result result = prepareSparseWrite(b, edit, new TypeArgument(this, typeArgs.clone()), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteTypedMap(edit, this, typeArgs.clone(), options, value.clone()); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedSet.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedSet.java index d61a55b..94c036b 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedSet.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedSet.java @@ -63,13 +63,13 @@ public final class LayoutTypedSet extends LayoutUniqueScope { public Result writeScope(RowBuffer b, RowCursor edit, TypeArgumentList typeArgs, UpdateOptions options, Out value) { Result result = prepareSparseWrite(b, edit, new TypeArgument(this, typeArgs.clone()), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteTypedSet(edit, this, typeArgs.clone(), options, value.clone()); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedTuple.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedTuple.java index dc30358..0bf72ab 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedTuple.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutTypedTuple.java @@ -75,13 +75,13 @@ public final class LayoutTypedTuple extends LayoutIndexedScope { public Result writeScope(RowBuffer b, RowCursor edit, TypeArgumentList typeArgs, UpdateOptions options, Out value) { Result result = LayoutType.prepareSparseWrite(b, edit, new TypeArgument(this, typeArgs.clone()), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteTypedTuple(edit, this, typeArgs.clone(), options, value.clone()); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUDT.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUDT.java index 5a98074..5d7fea3 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUDT.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUDT.java @@ -21,14 +21,14 @@ public final class LayoutUDT extends LayoutPropertyScope { } public int countTypeArgument(TypeArgumentList value) { - return (com.azure.data.cosmos.serialization.hybridrow.layouts.LayoutCode.SIZE / Byte.SIZE) + SchemaId.SIZE; + return (com.azure.data.cosmos.serialization.hybridrow.layouts.LayoutCode.SIZE / Byte.SIZE) + SchemaId.BYTES; } @Override public TypeArgumentList readTypeArgumentList(Reference row, int offset, Out lenInBytes) { SchemaId schemaId = row.get().readSchemaId(offset).clone(); - lenInBytes.setAndGet(SchemaId.SIZE); + lenInBytes.setAndGet(SchemaId.BYTES); return new TypeArgumentList(schemaId.clone()); } @@ -46,19 +46,19 @@ public final class LayoutUDT extends LayoutPropertyScope { TypeArgumentList typeArgs, UpdateOptions options, Out value) { Layout udt = b.get().resolver().resolve(typeArgs.schemaId().clone()); Result result = prepareSparseWrite(b, edit, new TypeArgument(this, typeArgs.clone()), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } b.get().WriteSparseUDT(edit, this, udt, options, value.clone()); - return Result.Success; + return Result.SUCCESS; } @Override public int writeTypeArgument(Reference row, int offset, TypeArgumentList value) { row.get().writeSparseTypeCode(offset, this.LayoutCode); row.get().writeSchemaId(offset + (com.azure.data.cosmos.serialization.hybridrow.layouts.LayoutCode.SIZE / Byte.SIZE), value.schemaId().clone()); - return (com.azure.data.cosmos.serialization.hybridrow.layouts.LayoutCode.SIZE / Byte.SIZE) + SchemaId.SIZE; + return (com.azure.data.cosmos.serialization.hybridrow.layouts.LayoutCode.SIZE / Byte.SIZE) + SchemaId.BYTES; } } \ No newline at end of file diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt16.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt16.java index 0122625..3f85707 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt16.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt16.java @@ -35,11 +35,11 @@ public final class LayoutUInt16 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadUInt16(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -48,13 +48,13 @@ public final class LayoutUInt16 extends LayoutType { public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseUInt16(edit)); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -65,12 +65,12 @@ public final class LayoutUInt16 extends LayoutType { short value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeUInt16(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -81,12 +81,12 @@ public final class LayoutUInt16 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, short value, UpdateOptions options) { Result result = prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseUInt16(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt32.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt32.java index 370821c..a5cc19e 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt32.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt32.java @@ -35,11 +35,11 @@ public final class LayoutUInt32 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadUInt32(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -48,13 +48,13 @@ public final class LayoutUInt32 extends LayoutType { public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseUInt32(edit)); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -65,12 +65,12 @@ public final class LayoutUInt32 extends LayoutType { int value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeUInt32(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -81,12 +81,12 @@ public final class LayoutUInt32 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, int value, UpdateOptions options) { Result result = prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseUInt32(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt64.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt64.java index f2cffc2..e3dfd47 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt64.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt64.java @@ -35,11 +35,11 @@ public final class LayoutUInt64 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadUInt64(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -48,13 +48,13 @@ public final class LayoutUInt64 extends LayoutType { public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseUInt64(edit)); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -65,12 +65,12 @@ public final class LayoutUInt64 extends LayoutType { long value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeUInt64(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -81,12 +81,12 @@ public final class LayoutUInt64 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, long value, UpdateOptions options) { Result result = prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseUInt64(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt8.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt8.java index 625aaa6..d0eb884 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt8.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUInt8.java @@ -35,11 +35,11 @@ public final class LayoutUInt8 extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().ReadUInt8(scope.get().start() + column.getOffset())); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -48,13 +48,13 @@ public final class LayoutUInt8 extends LayoutType { public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseUInt8(edit)); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -65,12 +65,12 @@ public final class LayoutUInt8 extends LayoutType { byte value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeUInt8(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -81,12 +81,12 @@ public final class LayoutUInt8 extends LayoutType { public Result WriteSparse(Reference b, Reference edit, byte value, UpdateOptions options) { Result result = prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseUInt8(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUniqueScope.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUniqueScope.java index 32caf10..bd10c50 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUniqueScope.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUniqueScope.java @@ -38,14 +38,14 @@ public abstract class LayoutUniqueScope extends LayoutIndexedScope { Reference patternScope, Out value) { Result result = LayoutType.prepareSparseMove(b, scope, this, this.FieldType(scope).clone(), patternScope, UpdateOptions.Update, value.clone()); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } // Check if the search found the result. b.get().DeleteSparse(patternScope); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -60,7 +60,7 @@ public abstract class LayoutUniqueScope extends LayoutIndexedScope { this.FieldType(destinationScope).clone(), sourceEdit, options, tempOut_dstEdit); dstEdit = tempOut_dstEdit.get(); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } @@ -73,7 +73,7 @@ public abstract class LayoutUniqueScope extends LayoutIndexedScope { // TODO: it would be "better" if the destinationScope were updated to point to the // highest item seen. Then we would avoid the maximum reparse. destinationScope.get().count(dstEdit.count()); - return Result.Success; + return Result.SUCCESS; } /** @@ -108,7 +108,7 @@ public abstract class LayoutUniqueScope extends LayoutIndexedScope { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = this.WriteScope(b, scope, typeArgs.clone(), out uniqueScope, options); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -120,9 +120,9 @@ public abstract class LayoutUniqueScope extends LayoutIndexedScope { Reference tempReference_childScope = new Reference(childScope); // TODO: C# TO JAVA CONVERTER: The following line could not be converted: - r = func == null ? null : func.Invoke(ref b, ref childScope, context) ??Result.Success; + r = func == null ? null : func.Invoke(ref b, ref childScope, context) ??Result.SUCCESS; childScope = tempReference_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { this.deleteScope(b, scope); return r; } @@ -132,7 +132,7 @@ public abstract class LayoutUniqueScope extends LayoutIndexedScope { new Reference(uniqueScope); r = b.get().TypedCollectionUniqueIndexRebuild(tempReference_uniqueScope); uniqueScope = tempReference_uniqueScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { this.deleteScope(b, scope); return r; } @@ -142,7 +142,7 @@ public abstract class LayoutUniqueScope extends LayoutIndexedScope { RowCursors.skip(scope.get().clone(), b, tempReference_childScope2); childScope = tempReference_childScope2.get(); - return Result.Success; + return Result.SUCCESS; } @Override diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUnixDateTime.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUnixDateTime.java index f747d93..7fe2bd3 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUnixDateTime.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutUnixDateTime.java @@ -31,24 +31,24 @@ public final class LayoutUnixDateTime extends LayoutType value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } value.setAndGet(b.get().ReadSparseUnixDateTime(edit).clone()); - return Result.Success; + return Result.SUCCESS; } @Override @@ -56,12 +56,12 @@ public final class LayoutUnixDateTime extends LayoutType implements ILayoutUtf8S // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = this.ReadFixed(b, scope, column, out span); - value.setAndGet((r == Result.Success) ? span.toString() :) + value.setAndGet((r == Result.SUCCESS) ? span.toString() :) default return r; } @@ -42,11 +42,11 @@ public final class LayoutUtf8 extends LayoutType implements ILayoutUtf8S checkArgument(col.getSize() >= 0); if (!b.get().readBit(scope.get().start(), col.getNullBit().clone())) { value.setAndGet(null); - return Result.NotFound; + return Result.NOT_FOUND; } value.setAndGet(b.get().readFixedString(scope.get().start() + col.getOffset(), col.getSize())); - return Result.Success; + return Result.SUCCESS; } @Override @@ -56,20 +56,20 @@ public final class LayoutUtf8 extends LayoutType implements ILayoutUtf8S // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = this.ReadSparse(b, edit, out span); - value.setAndGet((r == Result.Success) ? span.toString() :) + value.setAndGet((r == Result.SUCCESS) ? span.toString() :) default return r; } public Result ReadSparse(Reference b, Reference edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(null); return result; } value.setAndGet(b.get().readSparseString(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -79,7 +79,7 @@ public final class LayoutUtf8 extends LayoutType implements ILayoutUtf8S // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = this.ReadVariable(b, scope, column, out span); - value.setAndGet((r == Result.Success) ? span.toString() :) + value.setAndGet((r == Result.SUCCESS) ? span.toString() :) default return r; } @@ -89,13 +89,13 @@ public final class LayoutUtf8 extends LayoutType implements ILayoutUtf8S checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), col.getNullBit().clone())) { value.setAndGet(null); - return Result.NotFound; + return Result.NOT_FOUND; } int varOffset = b.get().ComputeVariableValueOffset(scope.get().layout(), scope.get().start(), col.getOffset()); value.setAndGet(b.get().readVariableString(varOffset)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -111,12 +111,12 @@ public final class LayoutUtf8 extends LayoutType implements ILayoutUtf8S checkArgument(col.getSize() >= 0); checkArgument(value.Length == col.getSize()); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } b.get().writeFixedString(scope.get().start() + col.getOffset(), value); b.get().setBit(scope.get().start(), col.getNullBit().clone()); - return Result.Success; + return Result.SUCCESS; } @Override @@ -145,12 +145,12 @@ public final class LayoutUtf8 extends LayoutType implements ILayoutUtf8S public Result WriteSparse(Reference b, Reference edit, Utf8Span value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseString(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override @@ -164,12 +164,12 @@ public final class LayoutUtf8 extends LayoutType implements ILayoutUtf8S LayoutColumn col, Utf8Span value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } int length = value.Length; if ((col.getSize() > 0) && (length > col.getSize())) { - return Result.TooBig; + return Result.TOO_BIG; } boolean exists = b.get().readBit(scope.get().start(), col.getNullBit().clone()); @@ -182,6 +182,6 @@ public final class LayoutUtf8 extends LayoutType implements ILayoutUtf8S b.get().setBit(scope.get().start(), col.getNullBit().clone()); scope.get().metaOffset(scope.get().metaOffset() + shift); scope.get().valueOffset(scope.get().valueOffset() + shift); - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutVarInt.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutVarInt.java index b694dc7..e71990e 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutVarInt.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutVarInt.java @@ -33,19 +33,19 @@ public final class LayoutVarInt extends LayoutType { Out value) { Contract.Fail("Not Implemented"); value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = LayoutType.prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().ReadSparseVarInt(edit)); - return Result.Success; + return Result.SUCCESS; } @Override @@ -53,20 +53,20 @@ public final class LayoutVarInt extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } int varOffset = b.get().computeVariableValueOffset(scope.get().layout(), scope.get().start(), column.getOffset()); value.setAndGet(b.get().ReadVariableInt(varOffset)); - return Result.Success; + return Result.SUCCESS; } @Override public Result WriteFixed(Reference b, Reference scope, LayoutColumn col, long value) { Contract.Fail("Not Implemented"); - return Result.Failure; + return Result.FAILURE; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -76,12 +76,12 @@ public final class LayoutVarInt extends LayoutType { public Result WriteSparse(Reference b, Reference edit, long value, UpdateOptions options) { Result result = LayoutType.prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseVarInt(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override @@ -94,7 +94,7 @@ public final class LayoutVarInt extends LayoutType { LayoutColumn col, long value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } boolean exists = b.get().readBit(scope.get().start(), col.getNullBit().clone()); @@ -107,6 +107,6 @@ public final class LayoutVarInt extends LayoutType { b.get().setBit(scope.get().start(), col.getNullBit().clone()); scope.get().metaOffset(scope.get().metaOffset() + shift); scope.get().valueOffset(scope.get().valueOffset() + shift); - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutVarUInt.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutVarUInt.java index c5d812c..bdcd51e 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutVarUInt.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/layouts/LayoutVarUInt.java @@ -38,7 +38,7 @@ public final class LayoutVarUInt extends LayoutType { Out value) { Contract.Fail("Not Implemented"); value.setAndGet(0); - return Result.Failure; + return Result.FAILURE; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -46,13 +46,13 @@ public final class LayoutVarUInt extends LayoutType { @Override public Result readSparse(RowBuffer b, RowCursor edit, Out value) { Result result = prepareSparseRead(b, edit, this.LayoutCode); - if (result != Result.Success) { + if (result != Result.SUCCESS) { value.setAndGet(0); return result; } value.setAndGet(b.get().readSparseVarUInt(edit)); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -63,13 +63,13 @@ public final class LayoutVarUInt extends LayoutType { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (!b.get().readBit(scope.get().start(), column.getNullBit().clone())) { value.setAndGet(0); - return Result.NotFound; + return Result.NOT_FOUND; } int varOffset = b.get().computeVariableValueOffset(scope.get().layout(), scope.get().start(), column.getOffset()); value.setAndGet(b.get().ReadVariableUInt(varOffset)); - return Result.Success; + return Result.SUCCESS; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -79,7 +79,7 @@ public final class LayoutVarUInt extends LayoutType { public Result WriteFixed(Reference b, Reference scope, LayoutColumn col, long value) { Contract.Fail("Not Implemented"); - return Result.Failure; + return Result.FAILURE; } //C# TO JAVA CONVERTER NOTE: Java does not support optional parameters. Overloaded method(s) are created above: @@ -90,12 +90,12 @@ public final class LayoutVarUInt extends LayoutType { public Result WriteSparse(Reference b, Reference edit, long value, UpdateOptions options) { Result result = prepareSparseWrite(b, edit, this.typeArg().clone(), options); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } b.get().WriteSparseVarUInt(edit, value, options); - return Result.Success; + return Result.SUCCESS; } @Override @@ -111,7 +111,7 @@ public final class LayoutVarUInt extends LayoutType { LayoutColumn col, long value) { checkArgument(scope.get().scopeType() instanceof LayoutUDT); if (scope.get().immutable()) { - return Result.InsufficientPermissions; + return Result.INSUFFICIENT_PERMISSIONS; } boolean exists = b.get().readBit(scope.get().start(), col.getNullBit().clone()); @@ -124,6 +124,6 @@ public final class LayoutVarUInt extends LayoutType { b.get().setBit(scope.get().start(), col.getNullBit().clone()); scope.get().metaOffset(scope.get().metaOffset() + shift); scope.get().valueOffset(scope.get().valueOffset() + shift); - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOFormatter.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOFormatter.java index eea166a..6d03ec1 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOFormatter.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOFormatter.java @@ -27,7 +27,7 @@ public final class RecordIOFormatter { //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: resizer = resizer != null ? resizer : DefaultSpanResizer.Default; resizer = resizer != null ? resizer : DefaultSpanResizer < Byte >.Default; - int estimatedSize = HybridRowHeader.SIZE + RecordIOFormatter.RecordLayout.getSize() + body.Length; + int estimatedSize = HybridRowHeader.BYTES + RecordIOFormatter.RecordLayout.getSize() + body.Length; //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: uint crc32 = Crc32.Update(0, body.Span); int crc32 = Crc32.Update(0, body.Span); @@ -49,7 +49,7 @@ public final class RecordIOFormatter { //ORIGINAL LINE: resizer = resizer != null ? resizer : DefaultSpanResizer.Default; resizer = resizer != null ? resizer : DefaultSpanResizer < Byte >.Default; int estimatedSize = - HybridRowHeader.SIZE + RecordIOFormatter.SegmentLayout.getSize() + segment.Comment == null ? null : + HybridRowHeader.BYTES + RecordIOFormatter.SegmentLayout.getSize() + segment.Comment == null ? null : segment.Comment.length() != null ? segment.Comment.length() : 0 + segment.SDL == null ? null : segment.SDL.length() != null ? segment.SDL.length() : 0 + 20; @@ -65,11 +65,11 @@ public final class RecordIOFormatter { row.setAndGet(new RowBuffer(initialCapacity, resizer)); row.get().initLayout(HybridRowVersion.V1, layout, SystemSchema.LayoutResolver); Result r = RowWriter.WriteBuffer(row.clone(), obj, writer); - if (r != Result.Success) { + if (r != Result.SUCCESS) { row.setAndGet(null); return r; } - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOParser.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOParser.java index 80ab268..927325b 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOParser.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOParser.java @@ -59,7 +59,7 @@ public final class RecordIOParser { public Result Process(Memory buffer, Out type, Out> record, Out need, Out consumed) { - Result r = Result.Failure; + Result r = Result.FAILURE; //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: Memory b = buffer; Memory b = buffer; @@ -70,11 +70,11 @@ public final class RecordIOParser { this.state = State.NeedSegmentLength; // TODO: C# TO JAVA CONVERTER: There is no 'goto' in Java: case NeedSegmentLength: { - int minimalSegmentRowSize = HybridRowHeader.SIZE + RecordIOFormatter.SegmentLayout.getSize(); + int minimalSegmentRowSize = HybridRowHeader.BYTES + RecordIOFormatter.SegmentLayout.getSize(); if (b.Length < minimalSegmentRowSize) { need.setAndGet(minimalSegmentRowSize); consumed.setAndGet(buffer.Length - b.Length); - return Result.InsufficientBuffer; + return Result.INSUFFICIENT_BUFFER; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -92,7 +92,7 @@ public final class RecordIOParser { r = SegmentSerializer.Read(tempReference_reader, tempOut_segment); this.segment = tempOut_segment.get(); reader = tempReference_reader.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { break; } @@ -105,7 +105,7 @@ public final class RecordIOParser { if (b.Length < this.segment.Length) { need.setAndGet(this.segment.Length); consumed.setAndGet(buffer.Length - b.Length); - return Result.InsufficientBuffer; + return Result.INSUFFICIENT_BUFFER; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -123,7 +123,7 @@ public final class RecordIOParser { r = SegmentSerializer.Read(tempReference_reader2, tempOut_segment2); this.segment = tempOut_segment2.get(); reader = tempReference_reader2.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { break; } @@ -133,14 +133,14 @@ public final class RecordIOParser { this.state = State.NeedHeader; consumed.setAndGet(buffer.Length - b.Length); type.setAndGet(ProductionType.Segment); - return Result.Success; + return Result.SUCCESS; } case NeedHeader: { - if (b.Length < HybridRowHeader.SIZE) { - need.setAndGet(HybridRowHeader.SIZE); + if (b.Length < HybridRowHeader.BYTES) { + need.setAndGet(HybridRowHeader.BYTES); consumed.setAndGet(buffer.Length - b.Length); - return Result.InsufficientBuffer; + return Result.INSUFFICIENT_BUFFER; } HybridRowHeader header; @@ -149,7 +149,7 @@ public final class RecordIOParser { // being modified: MemoryMarshal.TryRead(b.Span, out header); if (header.Version != HybridRowVersion.V1) { - r = Result.InvalidRow; + r = Result.INVALID_ROW; break; } @@ -165,16 +165,16 @@ public final class RecordIOParser { goto case State.NeedRecord } - r = Result.InvalidRow; + r = Result.INVALID_ROW; break; } case NeedRecord: { - int minimalRecordRowSize = HybridRowHeader.SIZE + RecordIOFormatter.RecordLayout.getSize(); + int minimalRecordRowSize = HybridRowHeader.BYTES + RecordIOFormatter.RecordLayout.getSize(); if (b.Length < minimalRecordRowSize) { need.setAndGet(minimalRecordRowSize); consumed.setAndGet(buffer.Length - b.Length); - return Result.InsufficientBuffer; + return Result.INSUFFICIENT_BUFFER; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: @@ -190,7 +190,7 @@ public final class RecordIOParser { r = RecordSerializer.Read(tempReference_reader3, tempOut_record); this.record = tempOut_record.get(); reader = tempReference_reader3.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { break; } @@ -204,7 +204,7 @@ public final class RecordIOParser { if (b.Length < this.record.Length) { need.setAndGet(this.record.Length); consumed.setAndGet(buffer.Length - b.Length); - return Result.InsufficientBuffer; + return Result.INSUFFICIENT_BUFFER; } record.setAndGet(b.Slice(0, this.record.Length)); @@ -214,7 +214,7 @@ public final class RecordIOParser { //ORIGINAL LINE: uint crc32 = Crc32.Update(0, record.Span); int crc32 = Crc32.Update(0, record.get().Span); if (crc32 != this.record.Crc32) { - r = Result.InvalidRow; + r = Result.INVALID_ROW; break; } @@ -223,7 +223,7 @@ public final class RecordIOParser { this.state = State.NeedHeader; consumed.setAndGet(buffer.Length - b.Length); type.setAndGet(ProductionType.Record); - return Result.Success; + return Result.SUCCESS; } } diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOStream.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOStream.java index 2bae0d9..1bc75ee 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOStream.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordIOStream.java @@ -123,7 +123,7 @@ public final class RecordIOStream { record = tempOut_record.get(); prodType = tempOut_prodType.get(); - if ((r != Result.Success) && (r != Result.InsufficientBuffer)) { + if ((r != Result.SUCCESS) && (r != Result.INSUFFICIENT_BUFFER)) { return r; } @@ -134,7 +134,7 @@ public final class RecordIOStream { } // If there wasn't enough data to move the parser forward then get more data. - if (r == Result.InsufficientBuffer) { + if (r == Result.INSUFFICIENT_BUFFER) { if (need > active.Length) { resizer.Resize(need, avail.Span); active = resizer.getMemory(); @@ -148,8 +148,8 @@ public final class RecordIOStream { if (prodType == RecordIOParser.ProductionType.Segment) { checkState(!record.IsEmpty); r = visitSegment == null ? null : visitSegment.invoke(record) != null ? - visitSegment.invoke(record) : Result.Success; - if (r != Result.Success) { + visitSegment.invoke(record) : Result.SUCCESS; + if (r != Result.SUCCESS) { return r; } } @@ -159,7 +159,7 @@ public final class RecordIOStream { checkState(!record.IsEmpty); r = visitRecord.invoke(record); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -168,7 +168,7 @@ public final class RecordIOStream { // Make sure we processed all of the available data. checkState(avail.Length == 0); - return Result.Success; + return Result.SUCCESS; } /** @@ -254,7 +254,7 @@ public final class RecordIOStream { //ORIGINAL LINE: Result r = RecordIOStream.FormatSegment(segment, resizer, out Memory metadata); Result r = RecordIOStream.FormatSegment(segment.clone(), resizer, tempOut_metadata); metadata = tempOut_metadata.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -268,7 +268,7 @@ public final class RecordIOStream { ReadOnlyMemory body; // TODO: C# TO JAVA CONVERTER: Java has no equivalent to the C# deconstruction assignments: (r, body) =await produce (index++); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -281,7 +281,7 @@ public final class RecordIOStream { //ORIGINAL LINE: r = RecordIOStream.FormatRow(body, resizer, out metadata); r = RecordIOStream.FormatRow(body, resizer, tempOut_metadata2); metadata = tempOut_metadata2.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -297,7 +297,7 @@ public final class RecordIOStream { await stm.WriteAsync(body); } - return Result.Success; + return Result.SUCCESS; } /** @@ -315,13 +315,13 @@ public final class RecordIOStream { Out tempOut_row = new Out(); Result r = RecordIOFormatter.FormatRecord(body, tempOut_row, resizer); row = tempOut_row.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { block.setAndGet(null); return r; } block.setAndGet(resizer.getMemory().Slice(0, row.Length)); - return Result.Success; + return Result.SUCCESS; } /** @@ -342,13 +342,13 @@ public final class RecordIOStream { new Out(); Result r = RecordIOFormatter.FormatSegment(segment.clone(), tempOut_row, resizer); row = tempOut_row.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { block.setAndGet(null); return r; } block.setAndGet(resizer.getMemory().Slice(0, row.Length)); - return Result.Success; + return Result.SUCCESS; } /** diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordSerializer.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordSerializer.java index 03a8f1a..806d2e7 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordSerializer.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/RecordSerializer.java @@ -19,7 +19,7 @@ public final class RecordSerializer { Out tempOut_Length = new Out(); r = reader.get().ReadInt32(tempOut_Length); obj.get().argValue.Length = tempOut_Length.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -30,7 +30,7 @@ public final class RecordSerializer { //ORIGINAL LINE: r = reader.ReadUInt32(out obj.Crc32); r = reader.get().ReadUInt32(tempOut_Crc32); obj.get().argValue.Crc32 = tempOut_Crc32.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -38,13 +38,13 @@ public final class RecordSerializer { } } - return Result.Success; + return Result.SUCCESS; } public static Result Write(Reference writer, TypeArgument typeArg, Record obj) { Result r; r = writer.get().WriteInt32("length", obj.Length); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } diff --git a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/SegmentSerializer.java b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/SegmentSerializer.java index 404dc93..2dc7519 100644 --- a/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/SegmentSerializer.java +++ b/java/src/main/java/com/azure/data/cosmos/serialization/hybridrow/recordio/SegmentSerializer.java @@ -37,14 +37,14 @@ public final class SegmentSerializer { Out tempOut_Length = new Out(); r = reader.get().ReadInt32(tempOut_Length); obj.get().argValue.Length = tempOut_Length.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } // If the RowBuffer isn't big enough to contain the rest of the header, then just // return the length. if (reader.get().getLength() < obj.get().Length) { - return Result.Success; + return Result.SUCCESS; } break; @@ -52,7 +52,7 @@ public final class SegmentSerializer { Out tempOut_Comment = new Out(); r = reader.get().ReadString(tempOut_Comment); obj.get().argValue.Comment = tempOut_Comment.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -61,7 +61,7 @@ public final class SegmentSerializer { Out tempOut_SDL = new Out(); r = reader.get().ReadString(tempOut_SDL); obj.get().argValue.SDL = tempOut_SDL.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -69,21 +69,21 @@ public final class SegmentSerializer { } } - return Result.Success; + return Result.SUCCESS; } public static Result Write(Reference writer, TypeArgument typeArg, Segment obj) { Result r; if (obj.Comment != null) { r = writer.get().WriteString("comment", obj.Comment); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } if (obj.SDL != null) { r = writer.get().WriteString("sdl", obj.SDL); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -93,11 +93,11 @@ public final class SegmentSerializer { // Because the length field is itself fixed, writing the length can never change the length. int length = writer.get().getLength(); r = writer.get().WriteInt32("length", length); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } checkState(length == writer.get().getLength()); - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/BenchmarkSuiteBase.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/BenchmarkSuiteBase.java index ece25fc..14da105 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/BenchmarkSuiteBase.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/BenchmarkSuiteBase.java @@ -111,19 +111,19 @@ public class BenchmarkSuiteBase { { body = null; if (index >= rows.size()) { - return Result.Success; + return Result.SUCCESS; } StreamingRowGenerator writer = new StreamingRowGenerator(BenchmarkSuiteBase.InitialCapacity, layout, resolver, resizer); Result r2 = writer.WriteBuffer(rows.get((int)index)); - if (r2 != Result.Success) { + if (r2 != Result.SUCCESS) { return r2; } body = resizer.getMemory().Slice(0, writer.getLength()); - return Result.Success; + return Result.SUCCESS; }); ResultAssert.IsSuccess(r); @@ -134,7 +134,7 @@ public class BenchmarkSuiteBase { //ORIGINAL LINE: private protected static class ResultAssert protected static class ResultAssert { public static void IsSuccess(Result actual) { - assert actual == Result.Success || Result.Success == actual; + assert actual == Result.SUCCESS || Result.SUCCESS == actual; } } } \ No newline at end of file diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/CodeGenRowGenerator.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/CodeGenRowGenerator.java index 35ab76d..369eb36 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/CodeGenRowGenerator.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/CodeGenRowGenerator.java @@ -101,7 +101,7 @@ public final class CodeGenRowGenerator { //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: public byte[] ToArray() public byte[] ToArray() { - return this.row.ToArray(); + return this.row.toArray(); } public Result WriteBuffer(HashMap tableValue) { @@ -164,21 +164,21 @@ public final class CodeGenRowGenerator { Utf8Span _; // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = LayoutType.Utf8.ReadVariable(row, root, this.street, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } Utf8Span _; // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these cannot be converted using the 'Out' helper class unless the method is within the code being modified: r = LayoutType.Utf8.ReadVariable(row, root, this.city, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } Utf8Span _; // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these cannot be converted using the 'Out' helper class unless the method is within the code being modified: r = LayoutType.Utf8.ReadFixed(row, root, this.state, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -187,21 +187,21 @@ public final class CodeGenRowGenerator { Out tempOut_childScope = new Out(); r = LayoutType.UDT.ReadScope(row, root, tempOut_childScope); childScope = tempOut_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } Reference tempReference_childScope = new Reference(childScope); r = this.postalCodeSerializer.ReadBuffer(row, tempReference_childScope); childScope = tempReference_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } Reference tempReference_childScope2 = new Reference(childScope); RowCursors.skip(root.get().clone(), row, tempReference_childScope2); childScope = tempReference_childScope2.get(); - return Result.Success; + return Result.SUCCESS; } @Override @@ -215,7 +215,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteVariable(row, root, this.street, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -227,7 +227,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteVariable(row, root, this.city, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -239,7 +239,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteFixed(row, root, this.state, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -256,14 +256,14 @@ public final class CodeGenRowGenerator { r = LayoutType.UDT.WriteScope(row, root, this.postalCode.typeArgs().clone(), tempOut_childScope); childScope = tempOut_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } Reference tempReference_childScope = new Reference(childScope); r = this.postalCodeSerializer.WriteBuffer(row, tempReference_childScope, (HashMap)value); childScope = tempReference_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -280,7 +280,7 @@ public final class CodeGenRowGenerator { } } - return Result.Success; + return Result.SUCCESS; } } @@ -364,7 +364,7 @@ public final class CodeGenRowGenerator { Out tempOut__ = new Out(); Result r = LayoutType.Guid.readFixed(row, root, this.guestId, tempOut__); _ = tempOut__.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -373,7 +373,7 @@ public final class CodeGenRowGenerator { // cannot be converted using the 'Out' helper class unless the method is within the code being // modified: r = LayoutType.Utf8.ReadVariable(row, root, this.firstName, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -382,7 +382,7 @@ public final class CodeGenRowGenerator { // cannot be converted using the 'Out' helper class unless the method is within the code being // modified: r = LayoutType.Utf8.ReadVariable(row, root, this.lastName, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -391,7 +391,7 @@ public final class CodeGenRowGenerator { // cannot be converted using the 'Out' helper class unless the method is within the code being // modified: r = LayoutType.Utf8.ReadVariable(row, root, this.title, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -401,7 +401,7 @@ public final class CodeGenRowGenerator { new Out(); r = LayoutType.TypedArray.ReadScope(row, root, tempOut_childScope); childScope = tempOut_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -414,7 +414,7 @@ public final class CodeGenRowGenerator { // these cannot be converted using the 'Ref' helper class unless the method is within the code // being modified: r = LayoutType.Utf8.ReadSparse(row, ref childScope, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -429,7 +429,7 @@ public final class CodeGenRowGenerator { new Out(); r = LayoutType.TypedArray.ReadScope(row, root, tempOut_childScope2); childScope = tempOut_childScope2.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -442,7 +442,7 @@ public final class CodeGenRowGenerator { // these cannot be converted using the 'Ref' helper class unless the method is within the code // being modified: r = LayoutType.Utf8.ReadSparse(row, ref childScope, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -457,7 +457,7 @@ public final class CodeGenRowGenerator { new Out(); r = LayoutType.TypedMap.ReadScope(row, root, tempOut_childScope3); childScope = tempOut_childScope3.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -470,12 +470,12 @@ public final class CodeGenRowGenerator { r = LayoutType.TypedTuple.ReadScope(row, tempReference_childScope3, tempOut_tupleScope); tupleScope = tempOut_tupleScope.get(); childScope = tempReference_childScope3.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } if (!tupleScope.MoveNext(row)) { - return Result.InvalidRow; + return Result.INVALID_ROW; } Utf8Span _; @@ -486,12 +486,12 @@ public final class CodeGenRowGenerator { // these cannot be converted using the 'Ref' helper class unless the method is within the code // being modified: r = LayoutType.Utf8.ReadSparse(row, ref tupleScope, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } if (!tupleScope.MoveNext(row)) { - return Result.InvalidRow; + return Result.INVALID_ROW; } Reference tempReference_tupleScope = @@ -502,7 +502,7 @@ public final class CodeGenRowGenerator { r = LayoutType.ImmutableUDT.ReadScope(row, tempReference_tupleScope, tempOut_valueScope); valueScope = tempOut_valueScope.get(); tupleScope = tempReference_tupleScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -510,7 +510,7 @@ public final class CodeGenRowGenerator { new Reference(valueScope); r = this.addressSerializer.ReadBuffer(row, tempReference_valueScope); valueScope = tempReference_valueScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -550,7 +550,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Guid.writeFixed(row, root, this.guestId, (UUID)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -563,7 +563,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteVariable(row, root, this.firstName, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -576,7 +576,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteVariable(row, root, this.lastName, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -589,7 +589,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteVariable(row, root, this.title, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -617,7 +617,7 @@ public final class CodeGenRowGenerator { (Utf8String)item); childScope = tempRef_childScope.argValue; row2 = tempRef_row2.argValue; - if (r2 != Result.Success) { + if (r2 != Result.SUCCESS) { return r2; } @@ -627,9 +627,9 @@ public final class CodeGenRowGenerator { row2 = tempRef_row22.argValue; } - return Result.Success; + return Result.SUCCESS; }); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -647,7 +647,7 @@ public final class CodeGenRowGenerator { r = LayoutType.TypedArray.WriteScope(row, root, this.phoneNumbers.typeArgs().clone(), tempOut_childScope); childScope = tempOut_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -656,7 +656,7 @@ public final class CodeGenRowGenerator { // 'ref' keyword - these cannot be converted using the 'Ref' helper class // unless the method is within the code being modified: r = LayoutType.Utf8.WriteSparse(row, ref childScope, (Utf8String)item); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -700,7 +700,7 @@ public final class CodeGenRowGenerator { (Utf8String)ctx2.value[0]); tupleScope = tempRef_tupleScope.argValue; row3 = tempRef_row3.argValue; - if (r3 != Result.Success) { + if (r3 != Result.SUCCESS) { return r3; } @@ -720,7 +720,7 @@ public final class CodeGenRowGenerator { }) childScope = tempReference_childScope.get(); row2 = tempReference_row2.get(); - if (r2 != Result.Success) { + if (r2 != Result.SUCCESS) { return r2; } @@ -730,9 +730,9 @@ public final class CodeGenRowGenerator { row2 = tempRef_row22.argValue; } - return Result.Success; + return Result.SUCCESS; }) - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -745,7 +745,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteVariable(row, root, this.confirmNumber, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -758,7 +758,7 @@ public final class CodeGenRowGenerator { } } - return Result.Success; + return Result.SUCCESS; } } @@ -806,7 +806,7 @@ public final class CodeGenRowGenerator { // cannot be converted using the 'Out' helper class unless the method is within the code being // modified: Result r = LayoutType.Utf8.ReadFixed(row, root, this.hotelId, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -815,7 +815,7 @@ public final class CodeGenRowGenerator { // cannot be converted using the 'Out' helper class unless the method is within the code being // modified: r = LayoutType.Utf8.ReadVariable(row, root, this.name, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -824,7 +824,7 @@ public final class CodeGenRowGenerator { // cannot be converted using the 'Out' helper class unless the method is within the code being // modified: r = LayoutType.Utf8.ReadVariable(row, root, this.phone, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -834,7 +834,7 @@ public final class CodeGenRowGenerator { new Out(); r = LayoutType.UDT.ReadScope(row, root, tempOut_childScope); childScope = tempOut_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -842,7 +842,7 @@ public final class CodeGenRowGenerator { new Reference(childScope); r = this.addressSerializer.ReadBuffer(row, tempReference_childScope); childScope = tempReference_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -851,7 +851,7 @@ public final class CodeGenRowGenerator { RowCursors.skip(root.get().clone(), row, tempReference_childScope2); childScope = tempReference_childScope2.get(); - return Result.Success; + return Result.SUCCESS; } @Override @@ -867,7 +867,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteFixed(row, root, this.hotelId, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -880,7 +880,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteVariable(row, root, this.name, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -893,7 +893,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteVariable(row, root, this.phone, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -911,7 +911,7 @@ public final class CodeGenRowGenerator { r = LayoutType.UDT.WriteScope(row, root, this.address.typeArgs().clone(), tempOut_childScope); childScope = tempOut_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -919,7 +919,7 @@ public final class CodeGenRowGenerator { r = this.addressSerializer.WriteBuffer(row, tempReference_childScope, (HashMap)value); childScope = tempReference_childScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -936,7 +936,7 @@ public final class CodeGenRowGenerator { } } - return Result.Success; + return Result.SUCCESS; } } @@ -975,7 +975,7 @@ public final class CodeGenRowGenerator { Out tempOut__ = new Out(); Result r = LayoutType.Int32.readFixed(row, root, this.zip, tempOut__); _ = tempOut__.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -1000,7 +1000,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Int32.WriteFixed(row, root, this.zip, (int)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -1014,7 +1014,7 @@ public final class CodeGenRowGenerator { if (value != null) { root.get().Find(row, this.plus4Token.clone()); r = LayoutType.Int16.WriteSparse(row, root, (short)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -1027,7 +1027,7 @@ public final class CodeGenRowGenerator { } } - return Result.Success; + return Result.SUCCESS; } } @@ -1066,7 +1066,7 @@ public final class CodeGenRowGenerator { // cannot be converted using the 'Out' helper class unless the method is within the code being // modified: Result r = LayoutType.Utf8.ReadFixed(row, root, this.hotelId, out _); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -1074,7 +1074,7 @@ public final class CodeGenRowGenerator { Out tempOut__ = new Out(); r = LayoutType.DateTime.ReadFixed(row, root, this.date, tempOut__); _ = tempOut__.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -1084,7 +1084,7 @@ public final class CodeGenRowGenerator { //ORIGINAL LINE: r = LayoutType.UInt8.ReadFixed(ref row, ref root, this.roomNumber, out byte _); r = LayoutType.UInt8.readFixed(row, root, this.roomNumber, tempOut__2); _ = tempOut__2.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -1108,7 +1108,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Utf8.WriteFixed(row, root, this.hotelId, (Utf8String)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -1121,7 +1121,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.DateTime.WriteFixed(row, root, this.date, (LocalDateTime)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -1137,7 +1137,7 @@ public final class CodeGenRowGenerator { //ORIGINAL LINE: r = LayoutType.UInt8.WriteFixed(ref row, ref root, this.roomNumber, // (byte)value); r = LayoutType.UInt8.WriteFixed(row, root, this.roomNumber, (byte)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -1150,7 +1150,7 @@ public final class CodeGenRowGenerator { case 0 if (value != null) { r = LayoutType.Boolean.WriteFixed(row, root, this.isAvailable, (boolean)value); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -1163,7 +1163,7 @@ public final class CodeGenRowGenerator { } } - return Result.Success; + return Result.SUCCESS; } } } \ No newline at end of file diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/JsonModelRowGenerator.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/JsonModelRowGenerator.java index 811463f..ad18c18 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/JsonModelRowGenerator.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/JsonModelRowGenerator.java @@ -72,7 +72,7 @@ public final class JsonModelRowGenerator { //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: public byte[] ToArray() public byte[] ToArray() { - return this.row.ToArray(); + return this.row.toArray(); } public Result WriteBuffer(HashMap value) { @@ -92,7 +92,7 @@ public final class JsonModelRowGenerator { return result; } - return Result.Success; + return Result.SUCCESS; }); this.row = tempReference_row.get(); return tempVar; @@ -166,7 +166,7 @@ public final class JsonModelRowGenerator { return result; } - return Result.Success; + return Result.SUCCESS; }); // TODO: C# TO JAVA CONVERTER: Java has no equivalent to C# pattern variables in 'case' statements: //ORIGINAL LINE: case List x: @@ -178,16 +178,16 @@ public final class JsonModelRowGenerator { Reference tempReference_writer2 = new Reference(writer2); Result result = JsonModelRowGenerator.JsonModelSwitch(tempReference_writer2, null, elm); writer2 = tempReference_writer2.get(); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } } - return Result.Success; + return Result.SUCCESS; }); default: throw new IllegalStateException(lenientFormat("Unknown type will be ignored: %s", value.getClass().getSimpleName())); - return Result.Failure; + return Result.FAILURE; } } } \ No newline at end of file diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/ReaderBenchmark.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/ReaderBenchmark.java index 33cba23..de12f84 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/ReaderBenchmark.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/ReaderBenchmark.java @@ -5,7 +5,6 @@ package com.azure.data.cosmos.serialization.hybridrow.perf; import com.azure.data.cosmos.core.Out; import com.azure.data.cosmos.core.Reference; -import com.azure.data.cosmos.core.Reference; import com.azure.data.cosmos.serialization.hybridrow.HybridRowVersion; import com.azure.data.cosmos.serialization.hybridrow.MemorySpanResizer; import com.azure.data.cosmos.serialization.hybridrow.Result; @@ -128,8 +127,8 @@ public final class ReaderBenchmark { { context.IncrementRecordCount(); r = ReaderBenchmark.VisitOneRow(record, context.getResolver()); - assert Result.Success == r; - return Result.Success; + assert Result.SUCCESS == r; + return Result.SUCCESS; }, segment -> { Segment _; @@ -137,13 +136,13 @@ public final class ReaderBenchmark { new Out(); r = SegmentSerializer.Read(segment.Span, context.getResolver(), tempOut__); _ = tempOut__.get(); - assert Result.Success == r; + assert Result.SUCCESS == r; // TODO: do something with embedded schema. - return Result.Success; + return Result.SUCCESS; }, resizer); - assert Result.Success == r; + assert Result.SUCCESS == r; } //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/RowReaderExtensions.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/RowReaderExtensions.java index ce0f562..5cc9a17 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/RowReaderExtensions.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/perf/RowReaderExtensions.java @@ -69,7 +69,7 @@ public final class RowReaderExtensions { case IMMUTABLE_TAGGED2_SCOPE: { // TODO: C# TO JAVA CONVERTER: The following lambda contained an unresolved 'ref' keyword - these are not converted by C# to Java Converter: Result r = reader.get().ReadScope(null, (ref RowReader child, Object _) -> child.VisitReader()); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -83,6 +83,6 @@ public final class RowReaderExtensions { } } - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/CustomerExampleUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/CustomerExampleUnitTests.java index e9ff31c..011d9ce 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/CustomerExampleUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/CustomerExampleUnitTests.java @@ -305,7 +305,7 @@ public final class CustomerExampleUnitTests { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = c.TypeAs().ReadScope(row, root, out addressScope); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -319,7 +319,7 @@ public final class CustomerExampleUnitTests { // cannot be converted using the 'Ref' helper class unless the method is within the code being // modified: r = c.TypeAs().WriteVariable(row, ref addressScope, c, a.Street); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -333,7 +333,7 @@ public final class CustomerExampleUnitTests { // cannot be converted using the 'Ref' helper class unless the method is within the code being // modified: r = c.TypeAs().WriteVariable(row, ref addressScope, c, a.City); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -347,7 +347,7 @@ public final class CustomerExampleUnitTests { // cannot be converted using the 'Ref' helper class unless the method is within the code being // modified: r = c.TypeAs().WriteFixed(row, ref addressScope, c, a.State); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -366,7 +366,7 @@ public final class CustomerExampleUnitTests { // cannot be converted using the 'Ref' helper class unless the method is within the code being // modified: r = c.TypeAs().WriteScope(row, ref addressScope, c.TypeArgs, out postalCodeScope); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -376,7 +376,7 @@ public final class CustomerExampleUnitTests { postalCodeScope = tempReference_postalCodeScope.get(); } - return Result.Success; + return Result.SUCCESS; } private int PrependGuestEmail(Reference row, Reference root, String email) { @@ -484,7 +484,7 @@ public final class CustomerExampleUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref emailScope, out emailScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref emailScope, out emailScope) == Result.SUCCESS) { g.Emails = new TreeSet(); while (emailScope.MoveNext(row)) { String item; @@ -511,7 +511,7 @@ public final class CustomerExampleUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref phoneNumbersScope, out phoneNumbersScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref phoneNumbersScope, out phoneNumbersScope) == Result.SUCCESS) { g.PhoneNumbers = new ArrayList(); while (phoneNumbersScope.MoveNext(row)) { String item; @@ -537,7 +537,7 @@ public final class CustomerExampleUnitTests { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref addressesScope, out addressesScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref addressesScope, out addressesScope) == Result.SUCCESS) { Reference tempReference_addressesScope = new Reference(addressesScope); TypeArgument tupleType = LayoutType.TypedMap.FieldType(tempReference_addressesScope).clone(); addressesScope = tempReference_addressesScope.get(); @@ -644,7 +644,7 @@ public final class CustomerExampleUnitTests { short plus4; // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadSparse(row, postalCodeScope, out plus4) == Result.Success) { + if (c.TypeAs().ReadSparse(row, postalCodeScope, out plus4) == Result.SUCCESS) { pc.Plus4 = plus4; } diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/LayoutCompilerUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/LayoutCompilerUnitTests.java index 34bfc14..b701435 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/LayoutCompilerUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/LayoutCompilerUnitTests.java @@ -2154,7 +2154,7 @@ private final static class RoundTripSparseObjectMulti extends TestActionDispatch // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: Result r = t.ReadSparse(row, ref nestedField, out value); - Assert.IsTrue(r == Result.NotFound || r == Result.TypeMismatch, tag); + Assert.IsTrue(r == Result.NOT_FOUND || r == Result.TYPE_MISMATCH, tag); // Write the nested field. // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these @@ -3215,7 +3215,7 @@ private final static class VariableInterleaving extends RoundTripVariable { Reference root, LayoutColumn col, Expected expected) { TLayout t = (TLayout)col.type(); Result r = t.writeVariable(row, root, col, (TValue)expected.TooBig); - Assert.AreEqual(Result.TooBig, r, "Json: {0}", expected.Json); + Assert.AreEqual(Result.TOO_BIG, r, "Json: {0}", expected.Json); } private , TValue> LayoutColumn Verify(Reference row, diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/NullableUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/NullableUnitTests.java index cb2b7be..72039f8 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/NullableUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/NullableUnitTests.java @@ -97,13 +97,13 @@ public final class NullableUnitTests { Result r = NullableUnitTests.ReadNullableImpl(row, scope, itemType.clone(), tempOut_value, nullableScope.clone()); value = tempOut_value.get(); - if ((r != Result.Success) && (r != Result.NotFound)) { + if ((r != Result.SUCCESS) && (r != Result.NOT_FOUND)) { item.setAndGet(null); return r; } - item.setAndGet((r == Result.NotFound) ? null : value); - return Result.Success; + item.setAndGet((r == Result.NOT_FOUND) ? null : value); + return Result.SUCCESS; } private static Result ReadNullable(Reference row, @@ -111,7 +111,7 @@ public final class NullableUnitTests { Out item, Out nullableScope) { Result r = NullableUnitTests.ReadNullableImpl(row, scope, itemType.clone(), item, nullableScope.clone()); - return (r == Result.NotFound) ? Result.Success : r; + return (r == Result.NOT_FOUND) ? Result.SUCCESS : r; } private static Result ReadNullableImpl(Reference row, @@ -119,7 +119,7 @@ public final class NullableUnitTests { Out item, Out nullableScope) { Result r = itemType.getType().TypeAs().ReadScope(row, scope, nullableScope.clone()); - if (r != Result.Success) { + if (r != Result.SUCCESS) { item.setAndGet(null); return r; } @@ -132,7 +132,7 @@ public final class NullableUnitTests { ResultAssert.NotFound(LayoutNullable.HasValue(row, nullableScope.clone())); item.setAndGet(null); - return Result.NotFound; + return Result.NOT_FOUND; } private Nullables ReadNullables(Reference row, Reference root) { @@ -150,7 +150,7 @@ public final class NullableUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.SUCCESS) { value.NullBool = new ArrayList(); RowCursor nullableScope = null; Reference tempReference_nullableScope = @@ -183,7 +183,7 @@ public final class NullableUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.SUCCESS) { value.NullArray = new ArrayList(); RowCursor nullableScope = null; Reference tempReference_nullableScope2 = @@ -216,7 +216,7 @@ public final class NullableUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.SUCCESS) { value.NullSet = new ArrayList(); RowCursor nullableScope = null; Reference tempReference_nullableScope3 = @@ -249,7 +249,7 @@ public final class NullableUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.SUCCESS) { value.NullTuple = new ArrayList<(Integer, Long) > (); RowCursor tupleScope = null; TypeArgument tupleType = c.TypeArgs[0]; @@ -316,7 +316,7 @@ public final class NullableUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref scope, out scope) == Result.SUCCESS) { //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: value.NullMap = new Dictionary>(); value.NullMap = new HashMap(); @@ -406,7 +406,7 @@ public final class NullableUnitTests { Result r = itemType.TypeAs().WriteScope(row, scope, itemType.getTypeArgs().clone(), hasValue, nullableScope); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -416,7 +416,7 @@ public final class NullableUnitTests { return r; } - return Result.Success; + return Result.SUCCESS; } private void WriteNullables(Reference row, Reference root, diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RecordIOUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RecordIOUnitTests.java index 7fbbaea..571bc86 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RecordIOUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RecordIOUnitTests.java @@ -98,7 +98,7 @@ public class RecordIOUnitTests { { body = null; if (index >= addresses.length) { - return Result.Success; + return Result.SUCCESS; } Out> tempOut_body = new Out>(); @@ -125,7 +125,7 @@ public class RecordIOUnitTests { obj = tempOut_obj.get(); ResultAssert.IsSuccess(r); addressesRead.add(obj); - return Result.Success; + return Result.SUCCESS; }, segment -> { assert !segment.IsEmpty; @@ -138,7 +138,7 @@ public class RecordIOUnitTests { ResultAssert.IsSuccess(r); assert obj.Comment == sampleComment; assert obj.SDL == sampleSDL; - return Result.Success; + return Result.SUCCESS; }, resizer); ResultAssert.IsSuccess(r); @@ -168,7 +168,7 @@ public class RecordIOUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = DiagnosticConverter.ReaderToString(tempReference_reader, out str); reader = tempReference_reader.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { obj.setAndGet(null); return r; } @@ -203,7 +203,7 @@ public class RecordIOUnitTests { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = DiagnosticConverter.ReaderToString(tempReference_reader, out str); reader = tempReference_reader.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { obj.setAndGet(null); return r; } @@ -231,12 +231,12 @@ public class RecordIOUnitTests { new Reference(row); Result r = RowWriter.WriteBuffer(tempReference_row, obj, AddressSerializer.Write); row = tempReference_row.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { buffer.setAndGet(null); return r; } buffer.setAndGet(resizer.getMemory().Slice(0, row.length())); - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/ResultAssert.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/ResultAssert.java index 99a91b3..b097139 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/ResultAssert.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/ResultAssert.java @@ -7,74 +7,74 @@ import com.azure.data.cosmos.serialization.hybridrow.Result; public final class ResultAssert { public static void Exists(Result actual) { - assert Result.Exists == actual; + assert Result.EXISTS == actual; } public static void Exists(Result actual, String message) { - Assert.AreEqual(Result.Exists, actual, message); + Assert.AreEqual(Result.EXISTS, actual, message); } public static void Exists(Result actual, String message, Object... parameters) { - Assert.AreEqual(Result.Exists, actual, message, parameters); + Assert.AreEqual(Result.EXISTS, actual, message, parameters); } public static void InsufficientPermissions(Result actual) { - assert Result.InsufficientPermissions == actual; + assert Result.INSUFFICIENT_PERMISSIONS == actual; } public static void InsufficientPermissions(Result actual, String message) { - Assert.AreEqual(Result.InsufficientPermissions, actual, message); + Assert.AreEqual(Result.INSUFFICIENT_PERMISSIONS, actual, message); } public static void InsufficientPermissions(Result actual, String message, Object... parameters) { - Assert.AreEqual(Result.InsufficientPermissions, actual, message, parameters); + Assert.AreEqual(Result.INSUFFICIENT_PERMISSIONS, actual, message, parameters); } public static void IsSuccess(Result actual) { - assert Result.Success == actual; + assert Result.SUCCESS == actual; } public static void IsSuccess(Result actual, String message) { - Assert.AreEqual(Result.Success, actual, message); + Assert.AreEqual(Result.SUCCESS, actual, message); } public static void IsSuccess(Result actual, String message, Object... parameters) { - Assert.AreEqual(Result.Success, actual, message, parameters); + Assert.AreEqual(Result.SUCCESS, actual, message, parameters); } public static void NotFound(Result actual) { - assert Result.NotFound == actual; + assert Result.NOT_FOUND == actual; } public static void NotFound(Result actual, String message) { - Assert.AreEqual(Result.NotFound, actual, message); + Assert.AreEqual(Result.NOT_FOUND, actual, message); } public static void NotFound(Result actual, String message, Object... parameters) { - Assert.AreEqual(Result.NotFound, actual, message, parameters); + Assert.AreEqual(Result.NOT_FOUND, actual, message, parameters); } public static void TypeConstraint(Result actual) { - assert Result.TypeConstraint == actual; + assert Result.TYPE_CONSTRAINT == actual; } public static void TypeConstraint(Result actual, String message) { - Assert.AreEqual(Result.TypeConstraint, actual, message); + Assert.AreEqual(Result.TYPE_CONSTRAINT, actual, message); } public static void TypeConstraint(Result actual, String message, Object... parameters) { - Assert.AreEqual(Result.TypeConstraint, actual, message, parameters); + Assert.AreEqual(Result.TYPE_CONSTRAINT, actual, message, parameters); } public static void TypeMismatch(Result actual) { - assert Result.TypeMismatch == actual; + assert Result.TYPE_MISMATCH == actual; } public static void TypeMismatch(Result actual, String message) { - Assert.AreEqual(Result.TypeMismatch, actual, message); + Assert.AreEqual(Result.TYPE_MISMATCH, actual, message); } public static void TypeMismatch(Result actual, String message, Object... parameters) { - Assert.AreEqual(Result.TypeMismatch, actual, message, parameters); + Assert.AreEqual(Result.TYPE_MISMATCH, actual, message, parameters); } } \ No newline at end of file diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RowReaderUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RowReaderUnitTests.java index d9ba6b5..fcdc12a 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RowReaderUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RowReaderUnitTests.java @@ -247,7 +247,7 @@ public final class RowReaderUnitTests { new Reference(nestedScope); Result result = rowReader.SkipScope(tempReference_nestedScope2); nestedScope = tempReference_nestedScope2.get(); - assert Result.Success != result; + assert Result.SUCCESS != result; } private static Result ReadNestedDocumentDelegate(Reference reader, int context) { @@ -265,7 +265,7 @@ public final class RowReaderUnitTests { } } - return Result.Success; + return Result.SUCCESS; } private static Result ReadNestedDocumentNonDelegate(Reference reader, int context) { @@ -292,7 +292,7 @@ public final class RowReaderUnitTests { } } - return Result.Success; + return Result.SUCCESS; } private static Result ReadNestedDocumentNonDelegateWithSkipScope(Reference reader, @@ -327,14 +327,14 @@ public final class RowReaderUnitTests { } } - return Result.Success; + return Result.SUCCESS; } private static Result ReadTuplePartial(Reference reader, int unused) { // Read only part of our tuple assert reader.get().Read(); assert reader.get().Read(); - return Result.Success; + return Result.SUCCESS; } private static Result WriteNestedDocument(Reference writer, TypeArgument typeArgument, @@ -357,7 +357,7 @@ public final class RowReaderUnitTests { if (level == 0) { ResultAssert.IsSuccess(writer.get().WriteScope("x", tupleArgument.clone(), 0, WriteTuple)); - return Result.Success; + return Result.SUCCESS; } ResultAssert.IsSuccess(writer.get().WriteScope("a", new TypeArgument(LayoutType.Object), level - 1, @@ -369,7 +369,7 @@ public final class RowReaderUnitTests { ResultAssert.IsSuccess(writer.get().WriteScope("c", new TypeArgument(LayoutType.Object), level - 1, RowReaderUnitTests.WriteNestedDocument)); - return Result.Success; + return Result.SUCCESS; } // TODO: C# TO JAVA CONVERTER: Java annotations will not correspond to .NET attributes: @@ -390,7 +390,7 @@ public final class RowReaderUnitTests { public Result Write(Reference writer, TypeArgument typeArg) { Result result = writer.get().WriteInt32("x", this.X); - if (result != Result.Success) { + if (result != Result.SUCCESS) { return result; } diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RowWriterUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RowWriterUnitTests.java index a6c8ce3..0343cbf 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RowWriterUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/RowWriterUnitTests.java @@ -137,7 +137,7 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer2.WriteInt8(null, value)); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col2 = @@ -158,11 +158,11 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer3.WriteFloat32(null, value2)); } - return Result.Success; + return Result.SUCCESS; })); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col3 = @@ -176,7 +176,7 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer2.WriteString(null, value)); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col4 = @@ -190,7 +190,7 @@ public final class RowWriterUnitTests { { ResultAssert.IsSuccess(writer2.WriteVarInt(null, values.Item1)); ResultAssert.IsSuccess(writer2.WriteInt64(null, values.Item2)); - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col5 = @@ -209,10 +209,10 @@ public final class RowWriterUnitTests { { ResultAssert.IsSuccess(writer3.WriteInt8(null, values2.Item1)); ResultAssert.IsSuccess(writer3.WriteInt8(null, values2.Item2)); - return Result.Success; + return Result.SUCCESS; })); - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col6 = @@ -230,7 +230,7 @@ public final class RowWriterUnitTests { (ref RowWriter writer3, TypeArgument typeArg2, IRowSerializable values2) -> values2.Write(tempReference_writer3, typeArg2.clone()))); writer3 = tempReference_writer3.get(); - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col7 = @@ -259,7 +259,7 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer2.WriteScope(null, typeArg.getTypeArgs().get(1).clone(), values.Item2, f1)); - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col8 = @@ -271,7 +271,7 @@ public final class RowWriterUnitTests { { ResultAssert.IsSuccess(writer2.WriteUInt8(null, values.Item1)); ResultAssert.IsSuccess(writer2.WriteString(null, values.Item2)); - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col9 = @@ -284,7 +284,7 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer2.WriteUInt8(null, values.Item1)); ResultAssert.IsSuccess(writer2.WriteBool(null, values.Item2)); ResultAssert.IsSuccess(writer2.WriteString(null, values.Item3)); - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col10 = @@ -298,7 +298,7 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer2.WriteString(null, value)); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col11 = @@ -320,11 +320,11 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer3.WriteInt8(null, value2)); } - return Result.Success; + return Result.SUCCESS; })); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col12 = @@ -346,11 +346,11 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer3.WriteInt32(null, value2)); } - return Result.Success; + return Result.SUCCESS; })); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col13 = @@ -373,7 +373,7 @@ public final class RowWriterUnitTests { writer3 = tempReference_writer3.get(); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col14 = @@ -392,11 +392,11 @@ public final class RowWriterUnitTests { { ResultAssert.IsSuccess(writer3.WriteString(null, values2.Item1)); ResultAssert.IsSuccess(writer3.WriteString(null, values2.Item2)); - return Result.Success; + return Result.SUCCESS; })); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col15 = @@ -421,14 +421,14 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer4.WriteInt8(null, value3)); } - return Result.Success; + return Result.SUCCESS; })); - return Result.Success; + return Result.SUCCESS; })); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col16 = @@ -457,18 +457,18 @@ public final class RowWriterUnitTests { { ResultAssert.IsSuccess(writer5.WriteInt32(null, values4.Item1)); ResultAssert.IsSuccess(writer5.WriteInt32(null, values4.Item2)); - return Result.Success; + return Result.SUCCESS; })); } - return Result.Success; + return Result.SUCCESS; })); - return Result.Success; + return Result.SUCCESS; })); } - return Result.Success; + return Result.SUCCESS; })); Out tempOut_col17 = @@ -485,16 +485,16 @@ public final class RowWriterUnitTests { ResultAssert.IsSuccess(writer3.WriteScope(null, typeArg2.getTypeArgs().get(1).clone(), values2.Item2, (ref RowWriter writer4, TypeArgument typeArg3, IRowSerializable values3) -> values3.Write(tempReference_writer4, typeArg3.clone()))); writer4 = tempReference_writer4.get(); - return Result.Success; + return Result.SUCCESS; })); } - return Result.Success; + return Result.SUCCESS; })); // Save the RowWriter length after everything is written for later comparison. writerLength = writer.Length; - return Result.Success; + return Result.SUCCESS; })); row = tempReference_row.get(); diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/SerializerUnitTest.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/SerializerUnitTest.java index fbe0ae0..f71458d 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/SerializerUnitTest.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/SerializerUnitTest.java @@ -70,7 +70,7 @@ public final class SerializerUnitTest { new Reference(row); Result r = RowWriter.WriteBuffer(tempReference_row, request, BatchRequestSerializer.Write); row = tempReference_row.get(); - assert Result.Success == r; + assert Result.SUCCESS == r; System.out.printf("Length of serialized row: %1$s" + "\r\n", row.length()); // Read the row back again. @@ -85,7 +85,7 @@ public final class SerializerUnitTest { r = BatchRequestSerializer.Read(tempReference_reader, tempOut__); _ = tempOut__.get(); reader = tempReference_reader.get(); - assert Result.Success == r; + assert Result.SUCCESS == r; // Dump the materialized request to the console. Reference tempReference_row3 = @@ -99,7 +99,7 @@ public final class SerializerUnitTest { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: r = DiagnosticConverter.ReaderToString(tempReference_reader2, out dumpStr); reader = tempReference_reader2.get(); - assert Result.Success == r; + assert Result.SUCCESS == r; System.out.println(dumpStr); } @@ -145,7 +145,7 @@ public final class SerializerUnitTest { Out tempOut_OperationType = new Out(); r = reader.get().ReadInt32(tempOut_OperationType); retval.OperationType = tempOut_OperationType.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -159,7 +159,7 @@ public final class SerializerUnitTest { (ref RowReader child, BatchOperation parent) -> BatchRequestHeadersSerializer.Read(tempReference_child, tempOut_Headers)); parent.Headers = tempOut_Headers.get(); child = tempReference_child.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -168,7 +168,7 @@ public final class SerializerUnitTest { Out tempOut_ResourceType = new Out(); r = reader.get().ReadInt32(tempOut_ResourceType); retval.ResourceType = tempOut_ResourceType.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -177,7 +177,7 @@ public final class SerializerUnitTest { Out tempOut_ResourcePath = new Out(); r = reader.get().ReadString(tempOut_ResourcePath); retval.ResourcePath = tempOut_ResourcePath.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -188,7 +188,7 @@ public final class SerializerUnitTest { //ORIGINAL LINE: r = reader.ReadBinary(out retval.ResourceBody); r = reader.get().ReadBinary(tempOut_ResourceBody); retval.ResourceBody = tempOut_ResourceBody.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -197,29 +197,29 @@ public final class SerializerUnitTest { } operation.setAndGet(retval); - return Result.Success; + return Result.SUCCESS; } public static Result Write(Reference writer, TypeArgument typeArg, BatchOperation operation) { Result r = writer.get().WriteInt32("operationType", operation.OperationType); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } r = writer.get().WriteScope("headers", BatchRequestHeadersSerializer.TypeArg, operation.Headers, BatchRequestHeadersSerializer.Write); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } r = writer.get().WriteInt32("resourceType", operation.ResourceType); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } r = writer.get().WriteString("resourcePath", operation.ResourcePath); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -250,7 +250,7 @@ public final class SerializerUnitTest { Out tempOut_SampleRequestHeader = new Out(); Result r = reader.get().ReadInt64(tempOut_SampleRequestHeader); retval.SampleRequestHeader = tempOut_SampleRequestHeader.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -259,7 +259,7 @@ public final class SerializerUnitTest { } header.setAndGet(retval); - return Result.Success; + return Result.SUCCESS; } public static Result Write(Reference writer, TypeArgument typeArg, @@ -280,7 +280,7 @@ public final class SerializerUnitTest { Out> tempOut_operations = new Out>(); Result r = RowReaderExtensions.ReadList(reader.get().clone(), BatchOperationSerializer.Read, tempOut_operations); operations = tempOut_operations.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { request.setAndGet(null); return r; } @@ -288,7 +288,7 @@ public final class SerializerUnitTest { request.setAndGet(new BatchRequest()); request.get().Operations = operations; - return Result.Success; + return Result.SUCCESS; } public static Result Write(Reference writer, TypeArgument typeArg, BatchRequest request) { @@ -297,12 +297,12 @@ public final class SerializerUnitTest { { for (BatchOperation operation : operations) { Result r = writer2.WriteScope(null, BatchOperationSerializer.TypeArg, operation, BatchOperationSerializer.Write); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } - return Result.Success; + return Result.SUCCESS; }); } } diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TaggedUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TaggedUnitTests.java index 98f0d56..b6ada00 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TaggedUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TaggedUnitTests.java @@ -83,7 +83,7 @@ public final class TaggedUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref tag1Scope, out tag1Scope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref tag1Scope, out tag1Scope) == Result.SUCCESS) { assert tag1Scope.Immutable; assert tag1Scope.MoveNext(row); byte apiCode; @@ -122,7 +122,7 @@ public final class TaggedUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref tag2Scope, out tag2Scope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref tag2Scope, out tag2Scope) == Result.SUCCESS) { assert !tag2Scope.Immutable; assert tag2Scope.MoveNext(row); byte apiCode; diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TupleUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TupleUnitTests.java index 3628341..d537a5f 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TupleUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TupleUnitTests.java @@ -238,18 +238,18 @@ public final class TupleUnitTests { row5 = tempReference_row55.get(); } - return Result.Success; + return Result.SUCCESS; })); tupCur = tempReference_tupCur2.get(); row4 = tempReference_row43.get(); } - return Result.Success; + return Result.SUCCESS; })); row3 = tempRef_row32.argValue; row3 = tempReference_row3.get(); - return Result.Success; + return Result.SUCCESS; })); arrCur = tempReference_arrCur.get(); row2 = tempReference_row2.get(); @@ -259,7 +259,7 @@ public final class TupleUnitTests { row2 = tempRef_row22.argValue; } - return Result.Success; + return Result.SUCCESS; })); history = tempReference_history.get(); row = tempReference_row3.get(); @@ -742,7 +742,7 @@ public final class TupleUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref valueScope, out valueScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref valueScope, out valueScope) == Result.SUCCESS) { assert valueScope.Immutable; assert valueScope.MoveNext(row); String units; @@ -778,7 +778,7 @@ public final class TupleUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref valueScope, out valueScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref valueScope, out valueScope) == Result.SUCCESS) { assert valueScope.Immutable; assert valueScope.MoveNext(row); String units; @@ -845,7 +845,7 @@ public final class TupleUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref valueScope, out valueScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref valueScope, out valueScope) == Result.SUCCESS) { assert valueScope.Immutable; assert valueScope.MoveNext(row); String units; diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedArrayUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedArrayUnitTests.java index 9142919..a604ada 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedArrayUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedArrayUnitTests.java @@ -128,7 +128,7 @@ public final class TypedArrayUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref tagsScope, out tagsScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref tagsScope, out tagsScope) == Result.SUCCESS) { value.Tags = new ArrayList(); while (tagsScope.MoveNext(row)) { String item; @@ -155,7 +155,7 @@ public final class TypedArrayUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref optionsScope, out optionsScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref optionsScope, out optionsScope) == Result.SUCCESS) { value.Options = new ArrayList(); while (optionsScope.MoveNext(row)) { TypeArgument itemType = c.TypeArgs[0]; @@ -203,7 +203,7 @@ public final class TypedArrayUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref ratingsScope, out ratingsScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref ratingsScope, out ratingsScope) == Result.SUCCESS) { value.Ratings = new ArrayList>(); TypeArgument innerType = c.TypeArgs[0]; LayoutTypedArray innerLayout = innerType.getType().TypeAs(); @@ -249,7 +249,7 @@ public final class TypedArrayUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref similarsScope, out similarsScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref similarsScope, out similarsScope) == Result.SUCCESS) { value.Similars = new ArrayList(); while (similarsScope.MoveNext(row)) { TypeArgument innerType = c.TypeArgs[0]; @@ -281,7 +281,7 @@ public final class TypedArrayUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.TypeAs().ReadScope(row, ref priorityScope, out priorityScope) == Result.Success) { + if (c.TypeAs().ReadScope(row, ref priorityScope, out priorityScope) == Result.SUCCESS) { value.Priority = new ArrayList>(); RowCursor tupleScope = null; Reference tempReference_tupleScope = diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedMapUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedMapUnitTests.java index ee1537a..1582b68 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedMapUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedMapUnitTests.java @@ -973,7 +973,7 @@ public final class TypedMapUnitTests { new Out(); Result r = tupleLayout.ReadScope(row, scope, tempOut_tupleScope); tupleScope = tempOut_tupleScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -985,7 +985,7 @@ public final class TypedMapUnitTests { r = typeArgs.get(0).getType().>TypeAs().ReadSparse(row, tempReference_tupleScope, tempOut_key); key = tempOut_key.get(); tupleScope = tempReference_tupleScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -997,12 +997,12 @@ public final class TypedMapUnitTests { r = typeArgs.get(1).getType().>TypeAs().ReadSparse(row, tempReference_tupleScope2, tempOut_value); value = tempOut_value.get(); tupleScope = tempReference_tupleScope2.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } pair.setAndGet(new KeyValuePair(key, value)); - return Result.Success; + return Result.SUCCESS; } private Movie ReadMovie(Reference row, Reference root) { @@ -1020,7 +1020,7 @@ public final class TypedMapUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref castScope, out castScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref castScope, out castScope) == Result.SUCCESS) { value.Cast = new HashMap(); while (castScope.MoveNext(row)) { Reference tempReference_castScope = @@ -1047,7 +1047,7 @@ public final class TypedMapUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref statsScope, out statsScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref statsScope, out statsScope) == Result.SUCCESS) { value.Stats = new HashMap(); while (statsScope.MoveNext(row)) { Reference tempReference_statsScope = @@ -1074,7 +1074,7 @@ public final class TypedMapUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref relatedScope, out relatedScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref relatedScope, out relatedScope) == Result.SUCCESS) { value.Related = new HashMap>(); TypeArgument keyType = c.TypeArgs[0]; TypeArgument valueType = c.TypeArgs[1]; @@ -1136,7 +1136,7 @@ public final class TypedMapUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref revenueScope, out revenueScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref revenueScope, out revenueScope) == Result.SUCCESS) { value.Revenue = new HashMap(); TypeArgument keyType = c.TypeArgs[0]; TypeArgument valueType = c.TypeArgs[1]; @@ -1195,7 +1195,7 @@ public final class TypedMapUnitTests { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: Result r = tupleLayout.WriteScope(row, scope, typeArgs.clone(), out tupleScope); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -1203,7 +1203,7 @@ public final class TypedMapUnitTests { new Reference(tupleScope); r = typeArgs.get(0).getType().>TypeAs().WriteSparse(row, tempReference_tupleScope, pair.Key); tupleScope = tempReference_tupleScope.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedSetUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedSetUnitTests.java index 920f05b..75d0897 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedSetUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/TypedSetUnitTests.java @@ -1195,7 +1195,7 @@ public final class TypedSetUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref tagsScope, out tagsScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref tagsScope, out tagsScope) == Result.SUCCESS) { value.Attendees = new ArrayList(); while (tagsScope.MoveNext(row)) { String item; @@ -1222,7 +1222,7 @@ public final class TypedSetUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref projScope, out projScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref projScope, out projScope) == Result.SUCCESS) { value.Projects = new ArrayList(); while (projScope.MoveNext(row)) { java.util.UUID item; @@ -1249,7 +1249,7 @@ public final class TypedSetUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref checkboxScope, out checkboxScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref checkboxScope, out checkboxScope) == Result.SUCCESS) { value.Checkboxes = new ArrayList(); while (checkboxScope.MoveNext(row)) { boolean item; @@ -1276,7 +1276,7 @@ public final class TypedSetUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref pricesScope, out pricesScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref pricesScope, out pricesScope) == Result.SUCCESS) { value.Prices = new ArrayList>(); TypeArgument innerType = c.TypeArgs[0]; LayoutUniqueScope innerLayout = innerType.getType().TypeAs(); @@ -1317,7 +1317,7 @@ public final class TypedSetUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref nestedScope, out nestedScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref nestedScope, out nestedScope) == Result.SUCCESS) { value.Nested = new ArrayList>>(); TypeArgument in2Type = c.TypeArgs[0]; LayoutUniqueScope in2Layout = in2Type.getType().TypeAs(); @@ -1372,7 +1372,7 @@ public final class TypedSetUnitTests { // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these // cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref shoppingScope, out shoppingScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref shoppingScope, out shoppingScope) == Result.SUCCESS) { value.Shopping = new ArrayList(); while (shoppingScope.MoveNext(row)) { TypeArgument innerType = c.TypeArgs[0]; @@ -1403,7 +1403,7 @@ public final class TypedSetUnitTests { // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'out' keyword - these // cannot be converted using the 'Out' helper class unless the method is within the code being modified: // TODO: C# TO JAVA CONVERTER: The following method call contained an unresolved 'ref' keyword - these cannot be converted using the 'Ref' helper class unless the method is within the code being modified: - if (c.typeAs().ReadScope(row, ref workScope, out workScope) == Result.Success) { + if (c.typeAs().ReadScope(row, ref workScope, out workScope) == Result.SUCCESS) { //C# TO JAVA CONVERTER WARNING: Unsigned integer types have no direct equivalent in Java: //ORIGINAL LINE: value.Work = new List>(); value.Work = new ArrayList>(); @@ -1456,12 +1456,12 @@ public final class TypedSetUnitTests { ResultAssert.IsSuccess(writer2.WriteGuid(null, item)); } - return Result.Success; + return Result.SUCCESS; }); return r; } - return Result.Success; + return Result.SUCCESS; } private static void WriteShoppingItem(Reference row, Reference matchScope, TypeArgumentList typeArgs, ShoppingItem m) { diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/UpdateOptionsUnitTests.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/UpdateOptionsUnitTests.java index df26c09..e29bd96 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/UpdateOptionsUnitTests.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/UpdateOptionsUnitTests.java @@ -11,10 +11,10 @@ public class UpdateOptionsUnitTests { // TODO: C# TO JAVA CONVERTER: Java annotations will not correspond to .NET attributes: //ORIGINAL LINE: [TestMethod][Owner("jthunter")] public void UpdateOptionsTest() public final void UpdateOptionsTest() { - assert RowOptions.None.getValue() == UpdateOptions.None.getValue(); - assert RowOptions.Update.getValue() == UpdateOptions.Update.getValue(); - assert RowOptions.Insert.getValue() == UpdateOptions.Insert.getValue(); - assert RowOptions.Upsert.getValue() == UpdateOptions.Upsert.getValue(); - assert RowOptions.InsertAt.getValue() == UpdateOptions.InsertAt.getValue(); + assert RowOptions.NONE.value() == UpdateOptions.None.getValue(); + assert RowOptions.UPDATE.value() == UpdateOptions.Update.getValue(); + assert RowOptions.INSERT.value() == UpdateOptions.Insert.getValue(); + assert RowOptions.UPSERT.value() == UpdateOptions.Upsert.getValue(); + assert RowOptions.INSERT_AT.value() == UpdateOptions.InsertAt.getValue(); } } \ No newline at end of file diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/customerschema/AddressSerializer.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/customerschema/AddressSerializer.java index 2911de4..4af88ac 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/customerschema/AddressSerializer.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/customerschema/AddressSerializer.java @@ -19,7 +19,7 @@ public final class AddressSerializer { Out tempOut_Street = new Out(); r = reader.get().ReadString(tempOut_Street); obj.get().argValue.Street = tempOut_Street.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -28,7 +28,7 @@ public final class AddressSerializer { Out tempOut_City = new Out(); r = reader.get().ReadString(tempOut_City); obj.get().argValue.City = tempOut_City.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -37,7 +37,7 @@ public final class AddressSerializer { Out tempOut_State = new Out(); r = reader.get().ReadString(tempOut_State); obj.get().argValue.State = tempOut_State.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -51,7 +51,7 @@ public final class AddressSerializer { parent.PostalCode = tempOut_PostalCode.get(); child = tempReference_child.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -59,28 +59,28 @@ public final class AddressSerializer { } } - return Result.Success; + return Result.SUCCESS; } public static Result Write(Reference writer, TypeArgument typeArg, Address obj) { Result r; if (obj.Street != null) { r = writer.get().WriteString("street", obj.Street); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } if (obj.City != null) { r = writer.get().WriteString("city", obj.City); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } if (obj.State != null) { r = writer.get().WriteString("state", obj.State); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } } @@ -91,6 +91,6 @@ public final class AddressSerializer { return r; } - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file diff --git a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/customerschema/PostalCodeSerializer.java b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/customerschema/PostalCodeSerializer.java index a360175..3fa8761 100644 --- a/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/customerschema/PostalCodeSerializer.java +++ b/java/src/test/java/com/azure/data/cosmos/serialization/hybridrow/unit/customerschema/PostalCodeSerializer.java @@ -30,7 +30,7 @@ public final class PostalCodeSerializer { Out tempOut_Zip = new Out(); r = reader.get().ReadInt32(tempOut_Zip); obj.get().argValue.Zip = tempOut_Zip.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -40,7 +40,7 @@ public final class PostalCodeSerializer { Out tempOut_value = new Out(); r = reader.get().ReadInt16(tempOut_value); value = tempOut_value.get(); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -49,13 +49,13 @@ public final class PostalCodeSerializer { } } - return Result.Success; + return Result.SUCCESS; } public static Result Write(Reference writer, TypeArgument typeArg, PostalCode obj) { Result r; r = writer.get().WriteInt32("zip", obj.Zip); - if (r != Result.Success) { + if (r != Result.SUCCESS) { return r; } @@ -64,6 +64,6 @@ public final class PostalCodeSerializer { return r; } - return Result.Success; + return Result.SUCCESS; } } \ No newline at end of file