Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id A632718E2E for ; Mon, 14 Dec 2015 06:00:21 +0000 (UTC) Received: (qmail 61567 invoked by uid 500); 14 Dec 2015 06:00:21 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 61528 invoked by uid 500); 14 Dec 2015 06:00:21 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 61212 invoked by uid 99); 14 Dec 2015 06:00:20 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 14 Dec 2015 06:00:20 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 9D1ECDFFCD; Mon, 14 Dec 2015 06:00:20 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Mon, 14 Dec 2015 06:00:30 -0000 Message-Id: <565c6758229b4be6b925294f2fc4ca42@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [11/29] ignite git commit: ignite-2065: rename "portable" classes to "binary" http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableContext.java deleted file mode 100644 index f7375a4..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableContext.java +++ /dev/null @@ -1,1102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.binary; - -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.binary.BinaryIdMapper; -import org.apache.ignite.binary.BinaryInvalidTypeException; -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinarySerializer; -import org.apache.ignite.binary.BinaryType; -import org.apache.ignite.binary.BinaryTypeConfiguration; -import org.apache.ignite.cache.CacheKeyConfiguration; -import org.apache.ignite.cache.affinity.AffinityKeyMapped; -import org.apache.ignite.configuration.BinaryConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.IgniteKernal; -import org.apache.ignite.internal.IgnitionEx; -import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; -import org.apache.ignite.internal.processors.datastructures.CollocatedQueueItemKey; -import org.apache.ignite.internal.processors.datastructures.CollocatedSetItemKey; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.internal.util.lang.GridMapEntry; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.marshaller.MarshallerContext; -import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; -import org.jetbrains.annotations.Nullable; -import org.jsr166.ConcurrentHashMap8; - -import java.io.Externalizable; -import java.io.File; -import java.io.IOException; -import java.io.InvalidObjectException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.io.ObjectStreamException; -import java.lang.reflect.Field; -import java.math.BigDecimal; -import java.net.URISyntaxException; -import java.net.URL; -import java.net.URLClassLoader; -import java.sql.Timestamp; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.Enumeration; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.LinkedList; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ConcurrentMap; -import java.util.jar.JarEntry; -import java.util.jar.JarFile; - -/** - * Portable context. - */ -public class PortableContext implements Externalizable { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - private static final ClassLoader dfltLdr = U.gridClassLoader(); - - /** */ - private final ConcurrentMap, PortableClassDescriptor> descByCls = new ConcurrentHashMap8<>(); - - /** Holds classes loaded by default class loader only. */ - private final ConcurrentMap userTypes = new ConcurrentHashMap8<>(); - - /** */ - private final Map predefinedTypes = new HashMap<>(); - - /** */ - private final Map predefinedTypeNames = new HashMap<>(); - - /** */ - private final Map, Byte> colTypes = new HashMap<>(); - - /** */ - private final Map, Byte> mapTypes = new HashMap<>(); - - /** */ - private final ConcurrentMap mappers = new ConcurrentHashMap8<>(0); - - /** Affinity key field names. */ - private final ConcurrentMap affKeyFieldNames = new ConcurrentHashMap8<>(0); - - /** */ - private final Map typeMappers = new ConcurrentHashMap8<>(0); - - /** */ - private BinaryMetadataHandler metaHnd; - - /** Actual marshaller. */ - private BinaryMarshaller marsh; - - /** */ - private MarshallerContext marshCtx; - - /** */ - private String gridName; - - /** */ - private IgniteConfiguration igniteCfg; - - /** */ - private final OptimizedMarshaller optmMarsh = new OptimizedMarshaller(); - - /** Compact footer flag. */ - private boolean compactFooter; - - /** Object schemas. */ - private volatile Map schemas; - - /** - * For {@link Externalizable}. - */ - public PortableContext() { - // No-op. - } - - /** - * @param metaHnd Meta data handler. - * @param igniteCfg Ignite configuration. - */ - public PortableContext(BinaryMetadataHandler metaHnd, IgniteConfiguration igniteCfg) { - assert metaHnd != null; - assert igniteCfg != null; - - this.metaHnd = metaHnd; - this.igniteCfg = igniteCfg; - - gridName = igniteCfg.getGridName(); - - colTypes.put(ArrayList.class, GridPortableMarshaller.ARR_LIST); - colTypes.put(LinkedList.class, GridPortableMarshaller.LINKED_LIST); - colTypes.put(HashSet.class, GridPortableMarshaller.HASH_SET); - colTypes.put(LinkedHashSet.class, GridPortableMarshaller.LINKED_HASH_SET); - - mapTypes.put(HashMap.class, GridPortableMarshaller.HASH_MAP); - mapTypes.put(LinkedHashMap.class, GridPortableMarshaller.LINKED_HASH_MAP); - - // IDs range from [0..200] is used by Java SDK API and GridGain legacy API - - registerPredefinedType(Byte.class, GridPortableMarshaller.BYTE); - registerPredefinedType(Boolean.class, GridPortableMarshaller.BOOLEAN); - registerPredefinedType(Short.class, GridPortableMarshaller.SHORT); - registerPredefinedType(Character.class, GridPortableMarshaller.CHAR); - registerPredefinedType(Integer.class, GridPortableMarshaller.INT); - registerPredefinedType(Long.class, GridPortableMarshaller.LONG); - registerPredefinedType(Float.class, GridPortableMarshaller.FLOAT); - registerPredefinedType(Double.class, GridPortableMarshaller.DOUBLE); - registerPredefinedType(String.class, GridPortableMarshaller.STRING); - registerPredefinedType(BigDecimal.class, GridPortableMarshaller.DECIMAL); - registerPredefinedType(Date.class, GridPortableMarshaller.DATE); - registerPredefinedType(Timestamp.class, GridPortableMarshaller.TIMESTAMP); - registerPredefinedType(UUID.class, GridPortableMarshaller.UUID); - - registerPredefinedType(byte[].class, GridPortableMarshaller.BYTE_ARR); - registerPredefinedType(short[].class, GridPortableMarshaller.SHORT_ARR); - registerPredefinedType(int[].class, GridPortableMarshaller.INT_ARR); - registerPredefinedType(long[].class, GridPortableMarshaller.LONG_ARR); - registerPredefinedType(float[].class, GridPortableMarshaller.FLOAT_ARR); - registerPredefinedType(double[].class, GridPortableMarshaller.DOUBLE_ARR); - registerPredefinedType(char[].class, GridPortableMarshaller.CHAR_ARR); - registerPredefinedType(boolean[].class, GridPortableMarshaller.BOOLEAN_ARR); - registerPredefinedType(BigDecimal[].class, GridPortableMarshaller.DECIMAL_ARR); - registerPredefinedType(String[].class, GridPortableMarshaller.STRING_ARR); - registerPredefinedType(UUID[].class, GridPortableMarshaller.UUID_ARR); - registerPredefinedType(Date[].class, GridPortableMarshaller.DATE_ARR); - registerPredefinedType(Timestamp[].class, GridPortableMarshaller.TIMESTAMP_ARR); - registerPredefinedType(Object[].class, GridPortableMarshaller.OBJ_ARR); - - registerPredefinedType(ArrayList.class, 0); - registerPredefinedType(LinkedList.class, 0); - registerPredefinedType(HashSet.class, 0); - registerPredefinedType(LinkedHashSet.class, 0); - - registerPredefinedType(HashMap.class, 0); - registerPredefinedType(LinkedHashMap.class, 0); - - registerPredefinedType(GridMapEntry.class, 60); - registerPredefinedType(IgniteBiTuple.class, 61); - registerPredefinedType(T2.class, 62); - - // IDs range [200..1000] is used by Ignite internal APIs. - } - - /** - * @return Marshaller. - */ - public BinaryMarshaller marshaller() { - return marsh; - } - - /** - * @return Ignite configuration. - */ - public IgniteConfiguration configuration(){ - return igniteCfg; - } - - /** - * @param marsh Portable marshaller. - * @param cfg Configuration. - * @throws BinaryObjectException In case of error. - */ - public void configure(BinaryMarshaller marsh, IgniteConfiguration cfg) throws BinaryObjectException { - if (marsh == null) - return; - - this.marsh = marsh; - - marshCtx = marsh.getContext(); - - BinaryConfiguration binaryCfg = cfg.getBinaryConfiguration(); - - if (binaryCfg == null) - binaryCfg = new BinaryConfiguration(); - - assert marshCtx != null; - - optmMarsh.setContext(marshCtx); - - configure( - binaryCfg.getIdMapper(), - binaryCfg.getSerializer(), - binaryCfg.getTypeConfigurations() - ); - - compactFooter = binaryCfg.isCompactFooter(); - } - - /** - * @param globalIdMapper ID mapper. - * @param globalSerializer Serializer. - * @param typeCfgs Type configurations. - * @throws BinaryObjectException In case of error. - */ - private void configure( - BinaryIdMapper globalIdMapper, - BinarySerializer globalSerializer, - Collection typeCfgs - ) throws BinaryObjectException { - TypeDescriptors descs = new TypeDescriptors(); - - Map affFields = new HashMap<>(); - - if (!F.isEmpty(igniteCfg.getCacheKeyConfiguration())) { - for (CacheKeyConfiguration keyCfg : igniteCfg.getCacheKeyConfiguration()) - affFields.put(keyCfg.getTypeName(), keyCfg.getAffinityKeyFieldName()); - } - - if (typeCfgs != null) { - for (BinaryTypeConfiguration typeCfg : typeCfgs) { - String clsName = typeCfg.getTypeName(); - - if (clsName == null) - throw new BinaryObjectException("Class name is required for portable type configuration."); - - BinaryIdMapper idMapper = globalIdMapper; - - if (typeCfg.getIdMapper() != null) - idMapper = typeCfg.getIdMapper(); - - idMapper = BinaryInternalIdMapper.create(idMapper); - - BinarySerializer serializer = globalSerializer; - - if (typeCfg.getSerializer() != null) - serializer = typeCfg.getSerializer(); - - if (clsName.endsWith(".*")) { - String pkgName = clsName.substring(0, clsName.length() - 2); - - for (String clsName0 : classesInPackage(pkgName)) - descs.add(clsName0, idMapper, serializer, affFields.get(clsName0), - typeCfg.isEnum(), true); - } - else - descs.add(clsName, idMapper, serializer, affFields.get(clsName), - typeCfg.isEnum(), false); - } - } - - for (TypeDescriptor desc : descs.descriptors()) - registerUserType(desc.clsName, desc.idMapper, desc.serializer, desc.affKeyFieldName, desc.isEnum); - - BinaryInternalIdMapper dfltMapper = BinaryInternalIdMapper.create(globalIdMapper); - - // Put affinity field names for unconfigured types. - for (Map.Entry entry : affFields.entrySet()) { - String typeName = entry.getKey(); - - int typeId = dfltMapper.typeId(typeName); - - affKeyFieldNames.putIfAbsent(typeId, entry.getValue()); - } - - addSystemClassAffinityKey(CollocatedSetItemKey.class); - addSystemClassAffinityKey(CollocatedQueueItemKey.class); - } - - /** - * @param cls Class. - */ - private void addSystemClassAffinityKey(Class cls) { - String fieldName = affinityFieldName(cls); - - assert fieldName != null : cls; - - affKeyFieldNames.putIfAbsent(cls.getName().hashCode(), affinityFieldName(cls)); - } - - /** - * @param pkgName Package name. - * @return Class names. - */ - @SuppressWarnings("ConstantConditions") - private static Iterable classesInPackage(String pkgName) { - assert pkgName != null; - - Collection clsNames = new ArrayList<>(); - - ClassLoader ldr = U.gridClassLoader(); - - if (ldr instanceof URLClassLoader) { - String pkgPath = pkgName.replaceAll("\\.", "/"); - - URL[] urls = ((URLClassLoader)ldr).getURLs(); - - for (URL url : urls) { - String proto = url.getProtocol().toLowerCase(); - - if ("file".equals(proto)) { - try { - File cpElement = new File(url.toURI()); - - if (cpElement.isDirectory()) { - File pkgDir = new File(cpElement, pkgPath); - - if (pkgDir.isDirectory()) { - for (File file : pkgDir.listFiles()) { - String fileName = file.getName(); - - if (file.isFile() && fileName.toLowerCase().endsWith(".class")) - clsNames.add(pkgName + '.' + fileName.substring(0, fileName.length() - 6)); - } - } - } - else if (cpElement.isFile()) { - try { - JarFile jar = new JarFile(cpElement); - - Enumeration entries = jar.entries(); - - while (entries.hasMoreElements()) { - String entry = entries.nextElement().getName(); - - if (entry.startsWith(pkgPath) && entry.endsWith(".class")) { - String clsName = entry.substring(pkgPath.length() + 1, entry.length() - 6); - - if (!clsName.contains("/") && !clsName.contains("\\")) - clsNames.add(pkgName + '.' + clsName); - } - } - } - catch (IOException ignored) { - // No-op. - } - } - } - catch (URISyntaxException ignored) { - // No-op. - } - } - } - } - - return clsNames; - } - - /** - * @param cls Class. - * @return Class descriptor. - * @throws BinaryObjectException In case of error. - */ - public PortableClassDescriptor descriptorForClass(Class cls, boolean deserialize) - throws BinaryObjectException { - assert cls != null; - - PortableClassDescriptor desc = descByCls.get(cls); - - if (desc == null || !desc.registered()) - desc = registerClassDescriptor(cls, deserialize); - - return desc; - } - - /** - * @param userType User type or not. - * @param typeId Type ID. - * @param ldr Class loader. - * @return Class descriptor. - */ - public PortableClassDescriptor descriptorForTypeId( - boolean userType, - int typeId, - ClassLoader ldr, - boolean deserialize - ) { - assert typeId != GridPortableMarshaller.UNREGISTERED_TYPE_ID; - - //TODO: As a workaround for IGNITE-1358 we always check the predefined map before without checking 'userType' - PortableClassDescriptor desc = predefinedTypes.get(typeId); - - if (desc != null) - return desc; - - if (ldr == null) - ldr = dfltLdr; - - // If the type hasn't been loaded by default class loader then we mustn't return the descriptor from here - // giving a chance to a custom class loader to reload type's class. - if (userType && ldr.equals(dfltLdr)) { - desc = userTypes.get(typeId); - - if (desc != null) - return desc; - } - - Class cls; - - try { - cls = marshCtx.getClass(typeId, ldr); - - desc = descByCls.get(cls); - } - catch (ClassNotFoundException e) { - // Class might have been loaded by default class loader. - if (userType && !ldr.equals(dfltLdr) && (desc = descriptorForTypeId(true, typeId, dfltLdr, deserialize)) != null) - return desc; - - throw new BinaryInvalidTypeException(e); - } - catch (IgniteCheckedException e) { - // Class might have been loaded by default class loader. - if (userType && !ldr.equals(dfltLdr) && (desc = descriptorForTypeId(true, typeId, dfltLdr, deserialize)) != null) - return desc; - - throw new BinaryObjectException("Failed resolve class for ID: " + typeId, e); - } - - if (desc == null) { - desc = registerClassDescriptor(cls, deserialize); - - assert desc.typeId() == typeId; - } - - return desc; - } - - /** - * Creates and registers {@link PortableClassDescriptor} for the given {@code class}. - * - * @param cls Class. - * @return Class descriptor. - */ - private PortableClassDescriptor registerClassDescriptor(Class cls, boolean deserialize) { - PortableClassDescriptor desc; - - String clsName = cls.getName(); - - if (marshCtx.isSystemType(clsName)) { - desc = new PortableClassDescriptor(this, - cls, - false, - clsName.hashCode(), - clsName, - null, - BinaryInternalIdMapper.defaultInstance(), - null, - false, - true, /* registered */ - false /* predefined */ - ); - - PortableClassDescriptor old = descByCls.putIfAbsent(cls, desc); - - if (old != null) - desc = old; - } - else - desc = registerUserClassDescriptor(cls, deserialize); - - return desc; - } - - /** - * Creates and registers {@link PortableClassDescriptor} for the given user {@code class}. - * - * @param cls Class. - * @return Class descriptor. - */ - private PortableClassDescriptor registerUserClassDescriptor(Class cls, boolean deserialize) { - boolean registered; - - String typeName = typeName(cls.getName()); - - BinaryIdMapper idMapper = userTypeIdMapper(typeName); - - int typeId = idMapper.typeId(typeName); - - try { - registered = marshCtx.registerClass(typeId, cls); - } - catch (IgniteCheckedException e) { - throw new BinaryObjectException("Failed to register class.", e); - } - - String affFieldName = affinityFieldName(cls); - - PortableClassDescriptor desc = new PortableClassDescriptor(this, - cls, - true, - typeId, - typeName, - affFieldName, - idMapper, - null, - true, - registered, - false /* predefined */ - ); - - if (!deserialize) { - Collection schemas = desc.schema() != null ? Collections.singleton(desc.schema()) : null; - - metaHnd.addMeta(typeId, - new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), affFieldName, schemas, desc.isEnum()).wrap(this)); - } - - // perform put() instead of putIfAbsent() because "registered" flag might have been changed or class loader - // might have reloaded described class. - if (IgniteUtils.detectClassLoader(cls).equals(dfltLdr)) - userTypes.put(typeId, desc); - - descByCls.put(cls, desc); - - mappers.putIfAbsent(typeId, idMapper); - - return desc; - } - - /** - * @param cls Collection class. - * @return Collection type ID. - */ - public byte collectionType(Class cls) { - assert cls != null; - - Byte type = colTypes.get(cls); - - if (type != null) - return type; - - return Set.class.isAssignableFrom(cls) ? GridPortableMarshaller.USER_SET : GridPortableMarshaller.USER_COL; - } - - /** - * @param cls Map class. - * @return Map type ID. - */ - public byte mapType(Class cls) { - assert cls != null; - - Byte type = mapTypes.get(cls); - - return type != null ? type : GridPortableMarshaller.USER_COL; - } - - /** - * @param typeName Type name. - * @return Type ID. - */ - public int typeId(String typeName) { - String typeName0 = typeName(typeName); - - Integer id = predefinedTypeNames.get(typeName0); - - if (id != null) - return id; - - if (marshCtx.isSystemType(typeName)) - return typeName.hashCode(); - - return userTypeIdMapper(typeName0).typeId(typeName0); - } - - /** - * @param typeId Type ID. - * @param fieldName Field name. - * @return Field ID. - */ - public int fieldId(int typeId, String fieldName) { - return userTypeIdMapper(typeId).fieldId(typeId, fieldName); - } - - /** - * @param typeId Type ID. - * @return Instance of ID mapper. - */ - public BinaryIdMapper userTypeIdMapper(int typeId) { - BinaryIdMapper idMapper = mappers.get(typeId); - - return idMapper != null ? idMapper : BinaryInternalIdMapper.defaultInstance(); - } - - /** - * @param typeName Type name. - * @return Instance of ID mapper. - */ - private BinaryIdMapper userTypeIdMapper(String typeName) { - BinaryIdMapper idMapper = typeMappers.get(typeName); - - return idMapper != null ? idMapper : BinaryInternalIdMapper.defaultInstance(); - } - - /** - * @param cls Class to get affinity field for. - * @return Affinity field name or {@code null} if field name was not found. - */ - private String affinityFieldName(Class cls) { - for (; cls != Object.class && cls != null; cls = cls.getSuperclass()) { - for (Field f : cls.getDeclaredFields()) { - if (f.getAnnotation(AffinityKeyMapped.class) != null) - return f.getName(); - } - } - - return null; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - U.writeString(out, igniteCfg.getGridName()); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - gridName = U.readString(in); - } - - /** - * @return Portable context. - * @throws ObjectStreamException In case of error. - */ - protected Object readResolve() throws ObjectStreamException { - try { - IgniteKernal g = IgnitionEx.gridx(gridName); - - if (g == null) - throw new IllegalStateException("Failed to find grid for name: " + gridName); - - return ((CacheObjectBinaryProcessorImpl)g.context().cacheObjects()).portableContext(); - } - catch (IllegalStateException e) { - throw U.withCause(new InvalidObjectException(e.getMessage()), e); - } - } - - /** - * @param cls Class. - * @param id Type ID. - * @return GridPortableClassDescriptor. - */ - public PortableClassDescriptor registerPredefinedType(Class cls, int id) { - String typeName = typeName(cls.getName()); - - PortableClassDescriptor desc = new PortableClassDescriptor( - this, - cls, - false, - id, - typeName, - null, - BinaryInternalIdMapper.defaultInstance(), - null, - false, - true, /* registered */ - true /* predefined */ - ); - - predefinedTypeNames.put(typeName, id); - predefinedTypes.put(id, desc); - - descByCls.put(cls, desc); - - return desc; - } - - /** - * @param clsName Class name. - * @param idMapper ID mapper. - * @param serializer Serializer. - * @param affKeyFieldName Affinity key field name. - * @param isEnum If enum. - * @throws BinaryObjectException In case of error. - */ - @SuppressWarnings("ErrorNotRethrown") - public void registerUserType(String clsName, - BinaryIdMapper idMapper, - @Nullable BinarySerializer serializer, - @Nullable String affKeyFieldName, - boolean isEnum) - throws BinaryObjectException { - assert idMapper != null; - - Class cls = null; - - try { - cls = Class.forName(clsName); - } - catch (ClassNotFoundException | NoClassDefFoundError ignored) { - // No-op. - } - - String typeName = typeName(clsName); - - int id = idMapper.typeId(typeName); - - //Workaround for IGNITE-1358 - if (predefinedTypes.get(id) != null) - throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']'); - - if (mappers.put(id, idMapper) != null) - throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']'); - - if (affKeyFieldName != null) { - if (affKeyFieldNames.put(id, affKeyFieldName) != null) - throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']'); - } - - typeMappers.put(typeName, idMapper); - - Map fieldsMeta = null; - Collection schemas = null; - - if (cls != null) { - PortableClassDescriptor desc = new PortableClassDescriptor( - this, - cls, - true, - id, - typeName, - affKeyFieldName, - idMapper, - serializer, - true, - true, /* registered */ - false /* predefined */ - ); - - fieldsMeta = desc.fieldsMeta(); - schemas = desc.schema() != null ? Collections.singleton(desc.schema()) : null; - - if (IgniteUtils.detectClassLoader(cls).equals(dfltLdr)) - userTypes.put(id, desc); - - descByCls.put(cls, desc); - } - - metaHnd.addMeta(id, new BinaryMetadata(id, typeName, fieldsMeta, affKeyFieldName, schemas, isEnum).wrap(this)); - } - - /** - * Create binary field. - * - * @param typeId Type ID. - * @param fieldName Field name. - * @return Binary field. - */ - public BinaryFieldImpl createField(int typeId, String fieldName) { - PortableSchemaRegistry schemaReg = schemaRegistry(typeId); - - int fieldId = userTypeIdMapper(typeId).fieldId(typeId, fieldName); - - return new BinaryFieldImpl(typeId, schemaReg, fieldName, fieldId); - } - - /** - * @param typeId Type ID. - * @return Meta data. - * @throws BinaryObjectException In case of error. - */ - @Nullable public BinaryType metadata(int typeId) throws BinaryObjectException { - return metaHnd != null ? metaHnd.metadata(typeId) : null; - } - - /** - * @param typeId Type ID. - * @return Affinity key field name. - */ - public String affinityKeyFieldName(int typeId) { - return affKeyFieldNames.get(typeId); - } - - /** - * @param typeId Type ID. - * @param meta Meta data. - * @throws BinaryObjectException In case of error. - */ - public void updateMetadata(int typeId, BinaryMetadata meta) throws BinaryObjectException { - metaHnd.addMeta(typeId, meta.wrap(this)); - } - - /** - * @return Whether field IDs should be skipped in footer or not. - */ - public boolean isCompactFooter() { - return compactFooter; - } - - /** - * Get schema registry for type ID. - * - * @param typeId Type ID. - * @return Schema registry for type ID. - */ - public PortableSchemaRegistry schemaRegistry(int typeId) { - Map schemas0 = schemas; - - if (schemas0 == null) { - synchronized (this) { - schemas0 = schemas; - - if (schemas0 == null) { - schemas0 = new HashMap<>(); - - PortableSchemaRegistry reg = new PortableSchemaRegistry(); - - schemas0.put(typeId, reg); - - schemas = schemas0; - - return reg; - } - } - } - - PortableSchemaRegistry reg = schemas0.get(typeId); - - if (reg == null) { - synchronized (this) { - reg = schemas.get(typeId); - - if (reg == null) { - reg = new PortableSchemaRegistry(); - - schemas0 = new HashMap<>(schemas); - - schemas0.put(typeId, reg); - - schemas = schemas0; - } - } - } - - return reg; - } - - /** - * Returns instance of {@link OptimizedMarshaller}. - * - * @return Optimized marshaller. - */ - OptimizedMarshaller optimizedMarsh() { - return optmMarsh; - } - - /** - * @param clsName Class name. - * @return Type name. - */ - @SuppressWarnings("ResultOfMethodCallIgnored") - public static String typeName(String clsName) { - assert clsName != null; - - int idx = clsName.lastIndexOf('$'); - - if (idx == clsName.length() - 1) - // This is a regular (not inner) class name that ends with '$'. Common use case for Scala classes. - idx = -1; - else if (idx >= 0) { - String typeName = clsName.substring(idx + 1); - - try { - Integer.parseInt(typeName); - - // This is an anonymous class. Don't cut off enclosing class name for it. - idx = -1; - } - catch (NumberFormatException ignore) { - // This is a lambda class. - if (clsName.indexOf("$$Lambda$") > 0) - idx = -1; - else - return typeName; - } - } - - if (idx < 0) - idx = clsName.lastIndexOf('.'); - - return idx >= 0 ? clsName.substring(idx + 1) : clsName; - } - - /** - * Undeployment callback invoked when class loader is being undeployed. - * - * Some marshallers may want to clean their internal state that uses the undeployed class loader somehow. - * - * @param ldr Class loader being undeployed. - */ - public void onUndeploy(ClassLoader ldr) { - for (Class cls : descByCls.keySet()) { - if (ldr.equals(cls.getClassLoader())) - descByCls.remove(cls); - } - - U.clearClassCache(ldr); - } - - /** - * Type descriptors. - */ - private static class TypeDescriptors { - /** Descriptors map. */ - private final Map descs = new LinkedHashMap<>(); - - /** - * Add type descriptor. - * - * @param clsName Class name. - * @param idMapper ID mapper. - * @param serializer Serializer. - * @param affKeyFieldName Affinity key field name. - * @param isEnum Enum flag. - * @param canOverride Whether this descriptor can be override. - * @throws BinaryObjectException If failed. - */ - private void add(String clsName, - BinaryIdMapper idMapper, - BinarySerializer serializer, - String affKeyFieldName, - boolean isEnum, - boolean canOverride) - throws BinaryObjectException { - TypeDescriptor desc = new TypeDescriptor(clsName, - idMapper, - serializer, - affKeyFieldName, - isEnum, - canOverride); - - TypeDescriptor oldDesc = descs.get(clsName); - - if (oldDesc == null) - descs.put(clsName, desc); - else - oldDesc.override(desc); - } - - /** - * Get all collected descriptors. - * - * @return Descriptors. - */ - private Iterable descriptors() { - return descs.values(); - } - } - - /** - * Type descriptor. - */ - private static class TypeDescriptor { - /** Class name. */ - private final String clsName; - - /** ID mapper. */ - private BinaryIdMapper idMapper; - - /** Serializer. */ - private BinarySerializer serializer; - - /** Affinity key field name. */ - private String affKeyFieldName; - - /** Enum flag. */ - private boolean isEnum; - - /** Whether this descriptor can be override. */ - private boolean canOverride; - - /** - * Constructor. - * - * @param clsName Class name. - * @param idMapper ID mapper. - * @param serializer Serializer. - * @param affKeyFieldName Affinity key field name. - * @param isEnum Enum type. - * @param canOverride Whether this descriptor can be override. - */ - private TypeDescriptor(String clsName, BinaryIdMapper idMapper, BinarySerializer serializer, - String affKeyFieldName, boolean isEnum, boolean canOverride) { - this.clsName = clsName; - this.idMapper = idMapper; - this.serializer = serializer; - this.affKeyFieldName = affKeyFieldName; - this.isEnum = isEnum; - this.canOverride = canOverride; - } - - /** - * Override portable class descriptor. - * - * @param other Other descriptor. - * @throws BinaryObjectException If failed. - */ - private void override(TypeDescriptor other) throws BinaryObjectException { - assert clsName.equals(other.clsName); - - if (canOverride) { - idMapper = other.idMapper; - serializer = other.serializer; - affKeyFieldName = other.affKeyFieldName; - canOverride = other.canOverride; - } - else if (!other.canOverride) - throw new BinaryObjectException("Duplicate explicit class definition in configuration: " + clsName); - } - } - - /** - * Type id wrapper. - */ - static class Type { - /** Type id */ - private final int id; - - /** Whether the following type is registered in a cache or not */ - private final boolean registered; - - /** - * @param id Id. - * @param registered Registered. - */ - public Type(int id, boolean registered) { - this.id = id; - this.registered = registered; - } - - /** - * @return Type ID. - */ - public int id() { - return id; - } - - /** - * @return Registered flag value. - */ - public boolean registered() { - return registered; - } - } -} http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/PortablePositionReadable.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortablePositionReadable.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/PortablePositionReadable.java deleted file mode 100644 index 8db6384..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortablePositionReadable.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.binary; - -/** - * Interface allowing for positioned read. - */ -public interface PortablePositionReadable { - /** - * Read byte at the given position. - * - * @param pos Position. - * @return Value. - */ - public byte readBytePositioned(int pos); - - /** - * Read short at the given position. - * - * @param pos Position. - * @return Value. - */ - public short readShortPositioned(int pos); - - /** - * Read integer at the given position. - * - * @param pos Position. - * @return Value. - */ - public int readIntPositioned(int pos); -} http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/PortablePrimitives.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortablePrimitives.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/PortablePrimitives.java deleted file mode 100644 index e5ff494..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortablePrimitives.java +++ /dev/null @@ -1,382 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.binary; - -import org.apache.ignite.internal.util.GridUnsafe; -import sun.misc.Unsafe; - -import java.nio.ByteOrder; - -/** - * Primitives writer. - */ -public abstract class PortablePrimitives { - /** */ - private static final Unsafe UNSAFE = GridUnsafe.unsafe(); - - /** */ - private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class); - - /** */ - private static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class); - - /** Whether little endian is set. */ - private static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN; - - /** - * @param arr Array. - * @param off Offset. - * @param val Value. - */ - public static void writeByte(byte[] arr, int off, byte val) { - UNSAFE.putByte(arr, BYTE_ARR_OFF + off, val); - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static byte readByte(byte[] arr, int off) { - return UNSAFE.getByte(arr, BYTE_ARR_OFF + off); - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static byte readByte(long ptr, int off) { - return UNSAFE.getByte(ptr + off); - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static byte[] readByteArray(byte[] arr, int off, int len) { - byte[] arr0 = new byte[len]; - - UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, BYTE_ARR_OFF, len); - - return arr0; - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static byte[] readByteArray(long ptr, int off, int len) { - byte[] arr0 = new byte[len]; - - UNSAFE.copyMemory(null, ptr + off, arr0, BYTE_ARR_OFF, len); - - return arr0; - } - - /** - * @param arr Array. - * @param off Offset. - * @param val Value. - */ - public static void writeBoolean(byte[] arr, int off, boolean val) { - writeByte(arr, off, val ? (byte)1 : (byte)0); - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static boolean readBoolean(byte[] arr, int off) { - return readByte(arr, off) == 1; - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static boolean readBoolean(long ptr, int off) { - return readByte(ptr, off) == 1; - } - - /** - * @param arr Array. - * @param off Offset. - * @param val Value. - */ - public static void writeShort(byte[] arr, int off, short val) { - if (BIG_ENDIAN) - val = Short.reverseBytes(val); - - UNSAFE.putShort(arr, BYTE_ARR_OFF + off, val); - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static short readShort(byte[] arr, int off) { - short val = UNSAFE.getShort(arr, BYTE_ARR_OFF + off); - - if (BIG_ENDIAN) - val = Short.reverseBytes(val); - - return val; - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static short readShort(long ptr, int off) { - short val = UNSAFE.getShort(ptr + off); - - if (BIG_ENDIAN) - val = Short.reverseBytes(val); - - return val; - } - - /** - * @param arr Array. - * @param off Offset. - * @param val Value. - */ - public static void writeChar(byte[] arr, int off, char val) { - if (BIG_ENDIAN) - val = Character.reverseBytes(val); - - UNSAFE.putChar(arr, BYTE_ARR_OFF + off, val); - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static char readChar(byte[] arr, int off) { - char val = UNSAFE.getChar(arr, BYTE_ARR_OFF + off); - - if (BIG_ENDIAN) - val = Character.reverseBytes(val); - - return val; - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static char readChar(long ptr, int off) { - char val = UNSAFE.getChar(ptr + off); - - if (BIG_ENDIAN) - val = Character.reverseBytes(val); - - return val; - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static char[] readCharArray(byte[] arr, int off, int len) { - char[] arr0 = new char[len]; - - UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, CHAR_ARR_OFF, len << 1); - - if (BIG_ENDIAN) { - for (int i = 0; i < len; i++) - arr0[i] = Character.reverseBytes(arr0[i]); - } - - return arr0; - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static char[] readCharArray(long ptr, int off, int len) { - char[] arr0 = new char[len]; - - UNSAFE.copyMemory(null, ptr + off, arr0, CHAR_ARR_OFF, len << 1); - - if (BIG_ENDIAN) { - for (int i = 0; i < len; i++) - arr0[i] = Character.reverseBytes(arr0[i]); - } - - return arr0; - } - - /** - * @param arr Array. - * @param off Offset. - * @param val Value. - */ - public static void writeInt(byte[] arr, int off, int val) { - if (BIG_ENDIAN) - val = Integer.reverseBytes(val); - - UNSAFE.putInt(arr, BYTE_ARR_OFF + off, val); - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static int readInt(byte[] arr, int off) { - int val = UNSAFE.getInt(arr, BYTE_ARR_OFF + off); - - if (BIG_ENDIAN) - val = Integer.reverseBytes(val); - - return val; - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static int readInt(long ptr, int off) { - int val = UNSAFE.getInt(ptr + off); - - if (BIG_ENDIAN) - val = Integer.reverseBytes(val); - - return val; - } - - /** - * @param arr Array. - * @param off Offset. - * @param val Value. - */ - public static void writeLong(byte[] arr, int off, long val) { - if (BIG_ENDIAN) - val = Long.reverseBytes(val); - - UNSAFE.putLong(arr, BYTE_ARR_OFF + off, val); - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static long readLong(byte[] arr, int off) { - long val = UNSAFE.getLong(arr, BYTE_ARR_OFF + off); - - if (BIG_ENDIAN) - val = Long.reverseBytes(val); - - return val; - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static long readLong(long ptr, int off) { - long val = UNSAFE.getLong(ptr + off); - - if (BIG_ENDIAN) - val = Long.reverseBytes(val); - - return val; - } - - /** - * @param arr Array. - * @param off Offset. - * @param val Value. - */ - public static void writeFloat(byte[] arr, int off, float val) { - int val0 = Float.floatToIntBits(val); - - writeInt(arr, off, val0); - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static float readFloat(byte[] arr, int off) { - int val = readInt(arr, off); - - return Float.intBitsToFloat(val); - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static float readFloat(long ptr, int off) { - int val = readInt(ptr, off); - - return Float.intBitsToFloat(val); - } - - /** - * @param arr Array. - * @param off Offset. - * @param val Value. - */ - public static void writeDouble(byte[] arr, int off, double val) { - long val0 = Double.doubleToLongBits(val); - - writeLong(arr, off, val0); - } - - /** - * @param arr Array. - * @param off Offset. - * @return Value. - */ - public static double readDouble(byte[] arr, int off) { - long val = readLong(arr, off); - - return Double.longBitsToDouble(val); - } - - /** - * @param ptr Pointer. - * @param off Offset. - * @return Value. - */ - public static double readDouble(long ptr, int off) { - long val = readLong(ptr, off); - - return Double.longBitsToDouble(val); - } -} http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableSchema.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableSchema.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableSchema.java deleted file mode 100644 index 61b5d45..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableSchema.java +++ /dev/null @@ -1,466 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.binary; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -/** - * Schema describing portable object content. We rely on the following assumptions: - * - When amount of fields in the object is low, it is better to inline these values into int fields thus allowing - * for quick comparisons performed within already fetched L1 cache line. - * - When there are more fields, we store them inside a hash map. - */ -public class PortableSchema implements Externalizable { - /** */ - private static final long serialVersionUID = 0L; - - /** Order returned if field is not found. */ - public static final int ORDER_NOT_FOUND = -1; - - /** Minimum sensible size. */ - private static final int MAP_MIN_SIZE = 32; - - /** Empty cell. */ - private static final int MAP_EMPTY = 0; - - /** Schema ID. */ - private int schemaId; - - /** IDs depending on order. */ - private int[] ids; - - /** Interned names of associated fields. */ - private String[] names; - - /** ID-to-order data. */ - private int[] idToOrderData; - - /** ID-to-order mask. */ - private int idToOrderMask; - - /** ID 1. */ - private int id0; - - /** ID 2. */ - private int id1; - - /** ID 3. */ - private int id2; - - /** ID 4. */ - private int id3; - - /** - * {@link Externalizable} support. - */ - public PortableSchema() { - // No-op. - } - - /** - * Constructor. - * - * @param schemaId Schema ID. - * @param fieldIds Field IDs. - */ - private PortableSchema(int schemaId, List fieldIds) { - assert fieldIds != null; - - this.schemaId = schemaId; - - initialize(fieldIds); - } - - /** - * @return Schema ID. - */ - public int schemaId() { - return schemaId; - } - - /** - * Try speculatively confirming order for the given field name. - * - * @param expOrder Expected order. - * @param expName Expected name. - * @return Field ID. - */ - @SuppressWarnings("StringEquality") - public Confirmation confirmOrder(int expOrder, String expName) { - assert expName != null; - - if (expOrder < names.length) { - String name = names[expOrder]; - - // Note that we use only reference equality assuming that field names are interned literals. - if (name == expName) - return Confirmation.CONFIRMED; - - if (name == null) - return Confirmation.CLARIFY; - } - - return Confirmation.REJECTED; - } - - /** - * Add field name. - * - * @param order Order. - * @param name Name. - */ - public void clarifyFieldName(int order, String name) { - assert name != null; - assert order < names.length; - - names[order] = name.intern(); - } - - /** - * Get field ID by order in footer. - * - * @param order Order. - * @return Field ID. - */ - public int fieldId(int order) { - return order < ids.length ? ids[order] : 0; - } - - /** - * Get field order in footer by field ID. - * - * @param id Field ID. - * @return Offset or {@code 0} if there is no such field. - */ - public int order(int id) { - if (idToOrderData == null) { - if (id == id0) - return 0; - - if (id == id1) - return 1; - - if (id == id2) - return 2; - - if (id == id3) - return 3; - - return ORDER_NOT_FOUND; - } - else { - int idx = (id & idToOrderMask) << 1; - - int curId = idToOrderData[idx]; - - if (id == curId) // Hit! - return idToOrderData[idx + 1]; - else if (curId == MAP_EMPTY) // No such ID! - return ORDER_NOT_FOUND; - else { - // Unlikely collision scenario. - for (int i = 2; i < idToOrderData.length; i += 2) { - int newIdx = (idx + i) % idToOrderData.length; - - assert newIdx < idToOrderData.length - 1; - - curId = idToOrderData[newIdx]; - - if (id == curId) - return idToOrderData[newIdx + 1]; - else if (curId == MAP_EMPTY) - return ORDER_NOT_FOUND; - } - - return ORDER_NOT_FOUND; - } - } - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - return schemaId; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - return o != null && o instanceof PortableSchema && schemaId == ((PortableSchema)o).schemaId; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeInt(schemaId); - - out.writeInt(ids.length); - - for (Integer id : ids) - out.writeInt(id); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - schemaId = in.readInt(); - - int idsCnt = in.readInt(); - - List fieldIds = new ArrayList<>(idsCnt); - - for (int i = 0; i < idsCnt; i++) - fieldIds.add(in.readInt()); - - initialize(fieldIds); - } - - /** - * Parse values. - * - * @param vals Values. - * @param size Proposed result size. - * @return Parse result. - */ - private static ParseResult parse(int[] vals, int size) { - int mask = maskForPowerOfTwo(size); - - int totalSize = size * 2; - - int[] data = new int[totalSize]; - int collisions = 0; - - for (int order = 0; order < vals.length; order++) { - int id = vals[order]; - - assert id != 0; - - int idIdx = (id & mask) << 1; - - if (data[idIdx] == 0) { - // Found empty slot. - data[idIdx] = id; - data[idIdx + 1] = order; - } - else { - // Collision! - collisions++; - - boolean placeFound = false; - - for (int i = 2; i < totalSize; i += 2) { - int newIdIdx = (idIdx + i) % totalSize; - - if (data[newIdIdx] == 0) { - data[newIdIdx] = id; - data[newIdIdx + 1] = order; - - placeFound = true; - - break; - } - } - - assert placeFound : "Should always have a place for entry!"; - } - } - - return new ParseResult(data, collisions); - } - - /** - * Get next power of two which greater or equal to the given number. - * This implementation is not meant to be very efficient, so it is expected to be used relatively rare. - * - * @param val Number - * @return Nearest pow2. - */ - private static int nextPowerOfTwo(int val) { - int res = 1; - - while (res < val) - res = res << 1; - - if (res < 0) - throw new IllegalArgumentException("Value is too big to find positive pow2: " + val); - - return res; - } - - /** - * Calculate mask for the given value which is a power of two. - * - * @param val Value. - * @return Mask. - */ - private static int maskForPowerOfTwo(int val) { - int mask = 0; - int comparand = 1; - - while (comparand < val) { - mask |= comparand; - - comparand <<= 1; - } - - return mask; - } - - /** - * Initialization routine. - * - * @param fieldIds Field IDs. - */ - private void initialize(List fieldIds) { - ids = new int[fieldIds.size()]; - - for (int i = 0; i < fieldIds.size(); i++) - ids[i] = fieldIds.get(i); - - names = new String[fieldIds.size()]; - - if (fieldIds.size() <= 4) { - Iterator iter = fieldIds.iterator(); - - id0 = iter.hasNext() ? iter.next() : 0; - id1 = iter.hasNext() ? iter.next() : 0; - id2 = iter.hasNext() ? iter.next() : 0; - id3 = iter.hasNext() ? iter.next() : 0; - } - else { - id0 = id1 = id2 = id3 = 0; - - initializeMap(ids); - } - } - - /** - * Initialize the map. - * - * @param vals Values. - */ - private void initializeMap(int[] vals) { - int size = Math.max(nextPowerOfTwo(vals.length) << 2, MAP_MIN_SIZE); - - assert size > 0; - - ParseResult finalRes; - - ParseResult res1 = parse(vals, size); - - if (res1.collisions == 0) - finalRes = res1; - else { - ParseResult res2 = parse(vals, size * 2); - - // Failed to decrease aom - if (res2.collisions == 0) - finalRes = res2; - else - finalRes = parse(vals, size * 4); - } - - idToOrderData = finalRes.data; - idToOrderMask = maskForPowerOfTwo(idToOrderData.length / 2); - } - - /** - * Schema builder. - */ - public static class Builder { - /** Schema ID. */ - private int schemaId = PortableUtils.schemaInitialId(); - - /** Fields. */ - private final ArrayList fields = new ArrayList<>(); - - /** - * Create new schema builder. - * - * @return Schema builder. - */ - public static Builder newBuilder() { - return new Builder(); - } - - /** - * Private constructor. - */ - private Builder() { - // No-op. - } - - /** - * Add field. - * - * @param fieldId Field ID. - */ - public void addField(int fieldId) { - fields.add(fieldId); - - schemaId = PortableUtils.updateSchemaId(schemaId, fieldId); - } - - /** - * Build schema. - * - * @return Schema. - */ - public PortableSchema build() { - return new PortableSchema(schemaId, fields); - } - } - - /** - * Order confirmation result. - */ - public enum Confirmation { - /** Confirmed. */ - CONFIRMED, - - /** Denied. */ - REJECTED, - - /** Field name clarification is needed. */ - CLARIFY - } - - /** - * Result of map parsing. - */ - private static class ParseResult { - /** Data. */ - private int[] data; - - /** Collisions. */ - private int collisions; - - /** - * Constructor. - * - * @param data Data. - * @param collisions Collisions. - */ - private ParseResult(int[] data, int collisions) { - this.data = data; - this.collisions = collisions; - } - } -} http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableSchemaRegistry.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableSchemaRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableSchemaRegistry.java deleted file mode 100644 index f3f92ee..0000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/PortableSchemaRegistry.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.binary; - -import org.jetbrains.annotations.Nullable; - -import java.util.HashMap; - -/** - * Portable schema registry. Contains all well-known object schemas. - *

- * We rely on the fact that usually object has only few different schemas. For this reason we inline several - * of them with optional fallback to normal hash map lookup. - * - */ -public class PortableSchemaRegistry { - /** Empty schema ID. */ - private static final int EMPTY = 0; - - /** Whether registry still works in inline mode. */ - private volatile boolean inline = true; - - /** First schema ID. */ - private int schemaId1; - - /** Second schema ID. */ - private int schemaId2; - - /** Third schema ID. */ - private int schemaId3; - - /** Fourth schema ID. */ - private int schemaId4; - - /** First schema. */ - private PortableSchema schema1; - - /** Second schema. */ - private PortableSchema schema2; - - /** Third schema. */ - private PortableSchema schema3; - - /** Fourth schema. */ - private PortableSchema schema4; - - /** Schemas with COW semantics. */ - private volatile HashMap schemas; - - /** - * Get schema for the given ID. We rely on very relaxed memory semantics here assuming that it is not critical - * to return false-positive {@code null} values. - * - * @param schemaId Schema ID. - * @return Schema or {@code null}. - */ - @Nullable public PortableSchema schema(int schemaId) { - if (inline) { - if (schemaId == schemaId1) - return schema1; - else if (schemaId == schemaId2) - return schema2; - else if (schemaId == schemaId3) - return schema3; - else if (schemaId == schemaId4) - return schema4; - } - else { - HashMap schemas0 = schemas; - - // Null can be observed here due to either data race or race condition when switching to non-inlined mode. - // Both of them are benign for us because they lead only to unnecessary schema re-calc. - if (schemas0 != null) - return schemas0.get(schemaId); - } - - return null; - } - - /** - * Add schema. - * - * @param schemaId Schema ID. - * @param schema Schema. - */ - public void addSchema(int schemaId, PortableSchema schema) { - synchronized (this) { - if (inline) { - // Check if this is already known schema. - if (schemaId == schemaId1 || schemaId == schemaId2 || schemaId == schemaId3 || schemaId == schemaId4) - return; - - // Try positioning new schema in inline mode. - if (schemaId1 == EMPTY) { - schemaId1 = schemaId; - - schema1 = schema; - - inline = true; // Forcing HB edge just in case. - - return; - } - - if (schemaId2 == EMPTY) { - schemaId2 = schemaId; - - schema2 = schema; - - inline = true; // Forcing HB edge just in case. - - return; - } - - if (schemaId3 == EMPTY) { - schemaId3 = schemaId; - - schema3 = schema; - - inline = true; // Forcing HB edge just in case. - - return; - } - - if (schemaId4 == EMPTY) { - schemaId4 = schemaId; - - schema4 = schema; - - inline = true; // Forcing HB edge just in case. - - return; - } - - // No luck, switching to hash map mode. - HashMap newSchemas = new HashMap<>(); - - newSchemas.put(schemaId1, schema1); - newSchemas.put(schemaId2, schema2); - newSchemas.put(schemaId3, schema3); - newSchemas.put(schemaId4, schema4); - - newSchemas.put(schemaId, schema); - - schemas = newSchemas; - - inline = false; - } - else { - HashMap newSchemas = new HashMap<>(schemas); - - newSchemas.put(schemaId, schema); - - schemas = newSchemas; - } - } - } -}