You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/12/21 11:43:08 UTC

[GitHub] [ignite-3] SammyVimes commented on a change in pull request #499: IGNITE-15721 Implemented the ability to use configuration instances as direct proxies to underlying storages.

SammyVimes commented on a change in pull request #499:
URL: https://github.com/apache/ignite-3/pull/499#discussion_r773057049



##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/DynamicProperty.java
##########
@@ -50,28 +61,40 @@
      * @param changer Configuration changer.
      * @param listenOnly Only adding listeners mode, without the ability to get or update the property value.
      * @param readOnly Value cannot be changed.
-     * @param injectedNameField Configuration field with {@link InjectedName}.
      */
     public DynamicProperty(
             List<String> prefix,
             String key,
             RootKey<?, ?> rootKey,
             DynamicConfigurationChanger changer,
             boolean listenOnly,
-            boolean readOnly,
-            boolean injectedNameField
+            boolean readOnly
     ) {
-        super(prefix, key, rootKey, changer, listenOnly);
+        super(
+                INJECTED_NAME.equals(key) || INTERNAL_ID.equals(key) ? prefix : appendKey(prefix, key),
+                key,
+                rootKey,
+                changer,
+                listenOnly
+        );
 
         this.readOnly = readOnly;
-        this.injectedNameField = injectedNameField;
+
+        this.injectedNameField = INJECTED_NAME.equals(key);
+        this.internalIdField = INTERNAL_ID.equals(key);
     }
 
     /** {@inheritDoc} */
     @Override
     public T value() {
         if (injectedNameField) {
+            // In this case "refreshValue()" is not of type "T", but an "InnerNode" holding it instead.

Review comment:
       I see that it's guarded, but external invariants can be broken (especially in large codebase such as our configuration), probably an assertion would make sense?  

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/DirectProxyAsmGenerator.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.configuration.asm;
+
+import static com.facebook.presto.bytecode.Access.FINAL;
+import static com.facebook.presto.bytecode.Access.PUBLIC;
+import static com.facebook.presto.bytecode.Parameter.arg;
+import static com.facebook.presto.bytecode.ParameterizedType.type;
+import static com.facebook.presto.bytecode.ParameterizedType.typeFromJavaClassName;
+import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantString;
+import static com.facebook.presto.bytecode.expression.BytecodeExpressions.invokeDynamic;
+import static com.facebook.presto.bytecode.expression.BytecodeExpressions.invokeStatic;
+import static com.facebook.presto.bytecode.expression.BytecodeExpressions.newInstance;
+import static java.lang.invoke.MethodType.methodType;
+import static java.util.Arrays.asList;
+import static java.util.EnumSet.of;
+import static org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator.LAMBDA_METAFACTORY;
+import static org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator.internalName;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isConfigValue;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isInjectedName;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isInternalId;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isNamedConfigValue;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isPolymorphicId;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isValue;
+import static org.apache.ignite.internal.util.CollectionUtils.concat;
+import static org.objectweb.asm.Opcodes.H_NEWINVOKESPECIAL;
+import static org.objectweb.asm.Type.VOID_TYPE;
+import static org.objectweb.asm.Type.getMethodDescriptor;
+import static org.objectweb.asm.Type.getMethodType;
+import static org.objectweb.asm.Type.getType;
+
+import com.facebook.presto.bytecode.BytecodeBlock;
+import com.facebook.presto.bytecode.ClassDefinition;
+import com.facebook.presto.bytecode.MethodDefinition;
+import com.facebook.presto.bytecode.ParameterizedType;
+import com.facebook.presto.bytecode.expression.BytecodeExpression;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.ConfigurationValue;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.annotation.InternalId;
+import org.apache.ignite.internal.configuration.DynamicConfigurationChanger;
+import org.apache.ignite.internal.configuration.direct.DirectConfigurationProxy;
+import org.apache.ignite.internal.configuration.direct.DirectNamedListProxy;
+import org.apache.ignite.internal.configuration.direct.DirectValueProxy;
+import org.apache.ignite.internal.configuration.direct.KeyPathNode;
+import org.apache.ignite.internal.configuration.tree.InnerNode;
+import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import org.objectweb.asm.Handle;
+
+/**
+ * Helper class to generate classes that extend {@link DirectConfigurationProxy}.
+ * All that's required here is to generate constructor and a bunch of getter methods.
+ */
+class DirectProxyAsmGenerator {
+    /** {@link DirectConfigurationProxy#DirectConfigurationProxy(List, DynamicConfigurationChanger)}. */
+    private static final Constructor<?> DIRECT_CFG_CTOR;
+
+    /** {@link ConfigurationUtil#appendKey(List, Object)}. */
+    private static final Method APPEND_KEY;
+
+    /** This generator instance is only used for {@link ConfigurationAsmGenerator#schemaInfo(java.lang.Class)}. */
+    private final ConfigurationAsmGenerator cgen;
+
+    /** Schema class. */
+    private final Class<?> schemaClass;
+
+    /** Set of internal extensions. */
+    private final Set<Class<?>> internalExtensions;
+
+    /** Fields from the schema class. */
+    private final List<Field> schemaFields;
+
+    /** Fields from all internal extensions. */
+    private final Collection<Field> internalExtensionsFields;
+
+    /** {@link InternalId} field. */
+    @Nullable
+    private final Field internalIdField;
+    private ClassDefinition classDef;
+
+    static {
+        try {
+            DIRECT_CFG_CTOR = DirectConfigurationProxy.class.getDeclaredConstructor(List.class, DynamicConfigurationChanger.class);
+
+            APPEND_KEY = ConfigurationUtil.class.getDeclaredMethod("appendKey", List.class, Object.class);
+        } catch (NoSuchMethodException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
+
+    /**
+     * Constructor.
+     * Please refer to indvidual fields for comments.
+     */
+    DirectProxyAsmGenerator(
+            ConfigurationAsmGenerator cgen,
+            Class<?> schemaClass,
+            Set<Class<?>> internalExtensions,
+            List<Field> schemaFields,
+            Collection<Field> internalExtensionsFields,
+            Field internalIdField
+    ) {
+        this.cgen = cgen;
+        this.schemaClass = schemaClass;
+        this.internalExtensions = internalExtensions;
+        this.schemaFields = schemaFields;
+        this.internalExtensionsFields = internalExtensionsFields;
+        this.internalIdField = internalIdField;
+    }
+
+    /**
+     * Generates class definition. Expected to be called once at most.
+     */
+    public ClassDefinition generate() {
+        assert classDef == null;
+
+        SchemaClassesInfo schemaClassInfo = cgen.schemaInfo(schemaClass);
+
+        // public final class FooDirectProxy extends DirectConfigurationProxy<Object, Object> implements FooConfiguration, ...
+        classDef = new ClassDefinition(
+                of(PUBLIC, FINAL),
+                internalName(schemaClassInfo.directProxyClassName),
+                type(DirectConfigurationProxy.class),
+                cgen.configClassInterfaces(schemaClass, internalExtensions)
+        );
+
+        addConstructor();
+
+        if (internalIdField != null) {
+            addGetMethod(internalIdField);
+        }
+
+        for (Field schemaField : concat(schemaFields, internalExtensionsFields)) {
+            addGetMethod(schemaField);
+        }
+
+        return classDef;
+    }
+
+    /**
+     * Generates constructor.
+     */
+    private void addConstructor() {
+        // public FooDirectProxy(List<KeyPathNode> keys, DynamicConfigurationChanger changer) {
+        MethodDefinition ctor = classDef.declareConstructor(
+                of(PUBLIC),
+                arg("keys", List.class),
+                arg("changer", DynamicConfigurationChanger.class)
+        );
+
+        //     super(keys, changer);
+        // }
+        ctor.getBody()
+                .append(ctor.getThis())
+                .append(ctor.getScope().getVariable("keys"))
+                .append(ctor.getScope().getVariable("changer"))
+                .invokeConstructor(DIRECT_CFG_CTOR)
+                .ret();
+    }
+
+    /**
+     * Generates getter based on the field.
+     */
+    private void addGetMethod(Field schemaField) {
+        Class<?> schemaFieldType = schemaField.getType();
+
+        String fieldName = schemaField.getName();
+
+        SchemaClassesInfo schemaClassInfo = cgen.schemaInfo(schemaFieldType);
+
+        ParameterizedType returnType;
+
+        // Return type is determined like in ConfigurationImpl class.
+        if (isConfigValue(schemaField)) {
+            returnType = typeFromJavaClassName(schemaClassInfo.cfgClassName);
+        } else if (isNamedConfigValue(schemaField)) {
+            returnType = type(NamedConfigurationTree.class);
+        } else {
+            assert isValue(schemaField) || isPolymorphicId(schemaField) || isInjectedName(schemaField)
+                    || isInternalId(schemaField) : schemaField;
+
+            returnType = type(ConfigurationValue.class);
+        }
+
+        MethodDefinition methodDef = classDef.declareMethod(
+                of(PUBLIC),
+                fieldName,
+                returnType
+        );
+
+        BytecodeBlock body = methodDef.getBody();
+
+        if (isValue(schemaField) || isPolymorphicId(schemaField) || isInjectedName(schemaField) || isInternalId(schemaField)) {
+            // new DirectValueProxy(appendKey(this.keys, new KeyPathNode("name")), changer);
+            // or
+            // new DirectValueProxy(appendKey(this.keys, new KeyPathNode("<internal_id>")), changer);
+            body.append(newInstance(
+                    DirectValueProxy.class,
+                    invokeStatic(
+                            APPEND_KEY,
+                            methodDef.getThis().getField("keys", List.class),
+                            newInstance(
+                                    KeyPathNode.class,
+                                    constantString(isInjectedName(schemaField) ? InnerNode.INJECTED_NAME
+                                            : isInternalId(schemaField) ? InnerNode.INTERNAL_ID : fieldName)
+                            )
+                    ),
+                    methodDef.getThis().getField("changer", DynamicConfigurationChanger.class)
+            ));
+        } else {
+            SchemaClassesInfo fieldSchemaClassInfo = cgen.schemaInfo(schemaField.getType());
+
+            ParameterizedType resultType = typeFromJavaClassName(fieldSchemaClassInfo.directProxyClassName);
+
+            if (isConfigValue(schemaField)) {
+                // new BarDirectProxy(appendKey(this.keys, new KeyPathNode("name")), changer);

Review comment:
       What's Bar?

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/DynamicProperty.java
##########
@@ -50,28 +61,40 @@
      * @param changer Configuration changer.
      * @param listenOnly Only adding listeners mode, without the ability to get or update the property value.
      * @param readOnly Value cannot be changed.
-     * @param injectedNameField Configuration field with {@link InjectedName}.
      */
     public DynamicProperty(
             List<String> prefix,
             String key,
             RootKey<?, ?> rootKey,
             DynamicConfigurationChanger changer,
             boolean listenOnly,
-            boolean readOnly,
-            boolean injectedNameField
+            boolean readOnly
     ) {
-        super(prefix, key, rootKey, changer, listenOnly);
+        super(
+                INJECTED_NAME.equals(key) || INTERNAL_ID.equals(key) ? prefix : appendKey(prefix, key),
+                key,
+                rootKey,
+                changer,
+                listenOnly
+        );
 
         this.readOnly = readOnly;
-        this.injectedNameField = injectedNameField;
+
+        this.injectedNameField = INJECTED_NAME.equals(key);
+        this.internalIdField = INTERNAL_ID.equals(key);
     }
 
     /** {@inheritDoc} */
     @Override
     public T value() {
         if (injectedNameField) {
+            // In this case "refreshValue()" is not of type "T", but an "InnerNode" holding it instead.
+            // "T" must be a String then, this is guarded by external invariants.
             return (T) ((InnerNode) refreshValue()).getInjectedNameFieldValue();
+        } else if (internalIdField) {
+            // In this case "refreshValue()" is not of type "T", but an "InnerNode" holding it instead.

Review comment:
       Same as above

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/DirectProxyAsmGenerator.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.configuration.asm;
+
+import static com.facebook.presto.bytecode.Access.FINAL;
+import static com.facebook.presto.bytecode.Access.PUBLIC;
+import static com.facebook.presto.bytecode.Parameter.arg;
+import static com.facebook.presto.bytecode.ParameterizedType.type;
+import static com.facebook.presto.bytecode.ParameterizedType.typeFromJavaClassName;
+import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantString;
+import static com.facebook.presto.bytecode.expression.BytecodeExpressions.invokeDynamic;
+import static com.facebook.presto.bytecode.expression.BytecodeExpressions.invokeStatic;
+import static com.facebook.presto.bytecode.expression.BytecodeExpressions.newInstance;
+import static java.lang.invoke.MethodType.methodType;
+import static java.util.Arrays.asList;
+import static java.util.EnumSet.of;
+import static org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator.LAMBDA_METAFACTORY;
+import static org.apache.ignite.internal.configuration.asm.ConfigurationAsmGenerator.internalName;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isConfigValue;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isInjectedName;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isInternalId;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isNamedConfigValue;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isPolymorphicId;
+import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isValue;
+import static org.apache.ignite.internal.util.CollectionUtils.concat;
+import static org.objectweb.asm.Opcodes.H_NEWINVOKESPECIAL;
+import static org.objectweb.asm.Type.VOID_TYPE;
+import static org.objectweb.asm.Type.getMethodDescriptor;
+import static org.objectweb.asm.Type.getMethodType;
+import static org.objectweb.asm.Type.getType;
+
+import com.facebook.presto.bytecode.BytecodeBlock;
+import com.facebook.presto.bytecode.ClassDefinition;
+import com.facebook.presto.bytecode.MethodDefinition;
+import com.facebook.presto.bytecode.ParameterizedType;
+import com.facebook.presto.bytecode.expression.BytecodeExpression;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.ConfigurationValue;
+import org.apache.ignite.configuration.NamedConfigurationTree;
+import org.apache.ignite.configuration.annotation.InternalId;
+import org.apache.ignite.internal.configuration.DynamicConfigurationChanger;
+import org.apache.ignite.internal.configuration.direct.DirectConfigurationProxy;
+import org.apache.ignite.internal.configuration.direct.DirectNamedListProxy;
+import org.apache.ignite.internal.configuration.direct.DirectValueProxy;
+import org.apache.ignite.internal.configuration.direct.KeyPathNode;
+import org.apache.ignite.internal.configuration.tree.InnerNode;
+import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import org.objectweb.asm.Handle;
+
+/**
+ * Helper class to generate classes that extend {@link DirectConfigurationProxy}.
+ * All that's required here is to generate constructor and a bunch of getter methods.
+ */
+class DirectProxyAsmGenerator {
+    /** {@link DirectConfigurationProxy#DirectConfigurationProxy(List, DynamicConfigurationChanger)}. */
+    private static final Constructor<?> DIRECT_CFG_CTOR;
+
+    /** {@link ConfigurationUtil#appendKey(List, Object)}. */
+    private static final Method APPEND_KEY;
+
+    /** This generator instance is only used for {@link ConfigurationAsmGenerator#schemaInfo(java.lang.Class)}. */
+    private final ConfigurationAsmGenerator cgen;
+
+    /** Schema class. */
+    private final Class<?> schemaClass;
+
+    /** Set of internal extensions. */
+    private final Set<Class<?>> internalExtensions;
+
+    /** Fields from the schema class. */
+    private final List<Field> schemaFields;
+
+    /** Fields from all internal extensions. */
+    private final Collection<Field> internalExtensionsFields;
+
+    /** {@link InternalId} field. */
+    @Nullable
+    private final Field internalIdField;
+    private ClassDefinition classDef;

Review comment:
       Missing newline and javadoc

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/ConfigurationNode.java
##########
@@ -72,20 +79,20 @@
     /**
      * Constructor.
      *
-     * @param prefix     Configuration prefix.
-     * @param key        Configuration key.
-     * @param rootKey    Root key.
-     * @param changer    Configuration changer.
+     * @param keys Configuration keys.

Review comment:
       I think this should be more elaborate

##########
File path: modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationUtil.java
##########
@@ -963,4 +1067,70 @@ public static boolean containsNameAnnotation(Field schemaField) {
             return List.copyOf(keys.subList(0, keys.size() - 1));
         }
     }
+
+    /**
+     * Returns a configuration tree for the purpose of reading configuration directly from the underlying storage. Actual reading is only
+     * happening while invoking {@link ConfigurationTree#value()}. It will either throw {@link NoSuchElementException},
+     * {@link StorageException} or return the value.
+     * <p/>
+     * It is important to understand how it processes named list elements. Imagine having element named {@code a} with internalId
+     * {@code aId}.
+     * <pre><code>
+     *     var namedListProxy = directProxy(namedList);
+     *
+     *     // Creates another proxy.
+     *     var aElementProxy = namedListProxy.get("a");
+     *
+     *     // This operation performs actual reading. It'll throw an exception if element named "a" doesn't exist anymore.
+     *     // It's been renamed or deleted.
+     *     var aElement = aElementProxy.value();
+     *
+     *     // Creates another proxy.
+     *     var aIdElementProxy = getByInternalId(namedListProxy, aId);
+     *
+     *     // This operation performs actual reading as previously stated. But, unlake the access by name, it won't throw an exception in

Review comment:
       ```suggestion
        *     // This operation performs actual reading as previously stated. But, unlike the access by name, it won't throw an exception in
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org