You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/02/16 23:35:54 UTC

[GitHub] [ozone] fapifta opened a new pull request #3104: Request validation

fapifta opened a new pull request #3104:
URL: https://github.com/apache/ozone/pull/3104


   ## What changes were proposed in this pull request?
   
   During developing different new features, the different protocols have to be kept in a stance where the old client can talk to a new server, and a new client also can talk to an old server.
   The latter was solved earlier by exposing the server version in the ServiceInfo, but the prior is a hard thing to deal with.
   
   After a few rounds of discussions we came up with an idea to implement a pluggable pre and post validation system, that can help us to either right away reject a specific query from an old client, or if the response contains something that the client can understand, give us the possibility to re-write the regular response to a form that the old client can understand, or at least give the old client a specific error message that says why the request fails.
   Similarly, the system has the possibility to adjust the request coming from the old clients if it is possible, so that the request can be served.
   
   Later on we have identified that there is a specific lifecycle phase, when the cluster is not finalized but already upgraded, certain requests in these cases might need to be right away cancelled, based on some request properties, but just until the new feature is not finalized.
   
   With this, we have created 4 conditions in which a validation can be activated (note that one validator can be activated by multiple conditions, and one condition can trigger multiple validators). These conditions are when a newer client posts a request, when an older client posts a request, when the cluster is in the pre-finalized state, and in case of every query a validator might need to run.
   
   Validators are assigned for request types, and request processing phases as well.
   Request types are as defined for the Ozone Manager protocol definition, while processing phases for now are pre and post processing.
   
   A post processor can not be a pre-processor at the same time, and for the different phases the validator method signature is different.
   Similarly, a validator can belong to only one request type.
   Binding happens at initialization time, when the Ozone Manager RPC server starts up.
   
   The validator methods are discovered based on the newly introduced RequestFeatureValidator annotation. Note that this annotation has a processor added as well, which checks requirements against annotated validation methods at compile time.
   
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6213
   
   ## How was this patch tested?
   JUnit tests added for the system, OM request handler code change is straightforward enough that is the only part for which I think tests are not necessary, let me know if I am wrong, and the current tests does not prove that the integration will work.
   


-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r809397531



##########
File path: dev-support/annotations/pom.xml
##########
@@ -0,0 +1,92 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+https://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <groupId>org.apache.ozone</groupId>
+  <artifactId>ozone-annotation-processing</artifactId>
+  <version>1.3.0-SNAPSHOT</version>
+  <description>Apache Ozone annotation processing tools for validating custom
+    annotations at compile time.
+  </description>
+  <name>Apache Ozone Annotation Processing</name>
+  <packaging>jar</packaging>
+
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    <spotbugs.version>3.1.12</spotbugs.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.github.spotbugs</groupId>
+      <artifactId>spotbugs</artifactId>
+      <version>${spotbugs.version}</version>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <!--
+       | It is required to add spotbugs dependency here, so that CI can check
+       | this package properly and does not fail the findbugs check.
+      -->
+      <plugin>
+        <groupId>com.github.spotbugs</groupId>
+        <artifactId>spotbugs-maven-plugin</artifactId>
+        <version>${spotbugs.version}</version>
+        <configuration>
+          <maxHeap>1024</maxHeap>
+          <xmlOutput>true</xmlOutput>
+          <includeTests>true</includeTests>
+        </configuration>
+      </plugin>
+      <!--
+       | Compiler plugin configuration have to be modified to compile the
+       | annotation processor, without having any annotation processor applied
+       | as due to the presence of
+       | META-INF/services/javac.annotation.processing.Processor the compilation
+       | would fail as the linked processor class can not be find while we are
+       | compiling it.
+      -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>3.9.0</version>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+        </configuration>
+        <executions>
+          <execution>
+            <id>default-compile</id>
+            <configuration>
+              <compilerArgument>-proc:none</compilerArgument>

Review comment:
       Thanks for noting this!, I have extended the note into the comment before the plugin definition about the compiler argument specified here.




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
errose28 commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r817115572



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidationCondition.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+
+import java.util.function.Predicate;
+
+/**
+ * Defines conditions for which validators can be assigned to.
+ *
+ * These conditions describe a situation where special request handling might
+ * be necessary. In these cases we do not override the actual request handling
+ * code, but based on certain request properties we might reject a request
+ * early, or we might modify the request, or the response received/sent from/to
+ * the client.
+ */
+public enum ValidationCondition {
+  /**
+   * Classifies validations that has to run after an upgrade until the cluster
+   * is in a pre-finalized state.
+   */
+  CLUSTER_NEEDS_FINALIZATION(r -> true,
+      ctx -> ctx.versionManager().needsFinalization()),
+  /**
+   * Classifies validations that has to run, when the client uses an older
+   * protocol version than the server.
+   */
+  OLDER_CLIENT_REQUESTS(r -> r.getVersion() < ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run, when the client uses a newer
+   * protocol version than the server.
+   */
+  NEWER_CLIENT_REQUESTS(r -> r.getVersion() > ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run for every request.
+   * If you plan to use this, please justify why the validation code should not
+   * be part of the actual request handling code.
+   */
+  UNCONDITIONAL(r -> true, ctx -> true);

Review comment:
       I don't think we need these last two right now. Maybe it would be better to remove them to reduce confusion? I can see the unconditional validator being misused since there isn't really a clearly defined use case. I think most newer client handling is going to happen on the client side based on the service info call, so having it here might be misleading for devs working on something with forward compatibility concerns.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidatorRegistry.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.reflections.Reflections;
+import org.reflections.scanners.MethodAnnotationsScanner;
+import org.reflections.util.ClasspathHelper;
+import org.reflections.util.ConfigurationBuilder;
+
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Registry that loads and stores the request validators to be applied by
+ * a service.
+ */
+public class ValidatorRegistry {
+
+  private final EnumMap<
+      ValidationCondition, EnumMap<Type, Pair<List<Method>, List<Method>>>>
+      validators = new EnumMap<>(ValidationCondition.class);
+
+  /**
+   * Creates a {@link ValidatorRegistry} instance that discovers validation
+   * methods in the provided package and the packages in the same resource.
+   * A validation method is recognized by the {@link RequestFeatureValidator}
+   * annotation that contains important information about how and when to use
+   * the validator.
+   * @param validatorPackage the main package inside which validatiors should
+   *                         be discovered.
+   */
+  ValidatorRegistry(String validatorPackage) {
+    this(ClasspathHelper.forPackage(validatorPackage));
+  }
+
+  /**
+   * Creates a {@link ValidatorRegistry} instance that discovers validation
+   * methods under the provided URL.
+   * A validation method is recognized by the {@link RequestFeatureValidator}
+   * annotation that contains important information about how and when to use
+   * the validator.
+   * @param searchUrls the path in which the annotated methods are searched.
+   */
+  ValidatorRegistry(Collection<URL> searchUrls) {
+    Reflections reflections = new Reflections(new ConfigurationBuilder()
+        .setUrls(searchUrls)
+        .setScanners(new MethodAnnotationsScanner())
+        .useParallelExecutor()
+    );
+
+    Set<Method> describedValidators =
+        reflections.getMethodsAnnotatedWith(RequestFeatureValidator.class);
+    initMaps(describedValidators);
+  }
+
+  /**
+   * Get the validators that has to be run in the given list of
+   * {@link ValidationCondition}s, for the given requestType and
+   * {@link RequestProcessingPhase}.
+   *
+   * @param conditions conditions that are present for the request
+   * @param requestType the type of the protocol message
+   * @param phase the request processing phase
+   * @return the list of validation methods that has to run.
+   */
+  List<Method> validationsFor(
+      List<ValidationCondition> conditions,
+      Type requestType,
+      RequestProcessingPhase phase) {
+
+    if (conditions.isEmpty() || validators.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    Set<Method> returnValue =
+        new HashSet<>(validationsFor(conditions.get(0), requestType, phase));

Review comment:
       How could we end up with duplicate method entries here that the set needs to remove?

##########
File path: dev-support/annotations/src/main/java/org/apache/ozone/annotations/RequestFeatureValidatorProcessor.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.annotations;
+
+import javax.annotation.processing.AbstractProcessor;
+import javax.annotation.processing.RoundEnvironment;
+import javax.annotation.processing.SupportedAnnotationTypes;
+import javax.annotation.processing.SupportedSourceVersion;
+import javax.lang.model.SourceVersion;
+import javax.lang.model.element.AnnotationMirror;
+import javax.lang.model.element.AnnotationValue;
+import javax.lang.model.element.AnnotationValueVisitor;
+import javax.lang.model.element.Element;
+import javax.lang.model.element.ElementKind;
+import javax.lang.model.element.ExecutableElement;
+import javax.lang.model.element.Modifier;
+import javax.lang.model.element.TypeElement;
+import javax.lang.model.element.VariableElement;
+import javax.lang.model.type.ExecutableType;
+import javax.lang.model.type.TypeMirror;
+import javax.lang.model.util.SimpleAnnotationValueVisitor8;
+import javax.tools.Diagnostic;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+
+@SupportedAnnotationTypes(
+    "org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator")
+@SupportedSourceVersion(SourceVersion.RELEASE_8)
+public class RequestFeatureValidatorProcessor extends AbstractProcessor {

Review comment:
       Can we add a javadoc to this class explaining it is used to validate annotations are constructed correctly, which annotations it is operating on, and that it is picked up from the javax.annotations.processing.Processor file?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidationCondition.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+
+import java.util.function.Predicate;
+
+/**
+ * Defines conditions for which validators can be assigned to.
+ *
+ * These conditions describe a situation where special request handling might
+ * be necessary. In these cases we do not override the actual request handling
+ * code, but based on certain request properties we might reject a request
+ * early, or we might modify the request, or the response received/sent from/to
+ * the client.
+ */
+public enum ValidationCondition {
+  /**
+   * Classifies validations that has to run after an upgrade until the cluster
+   * is in a pre-finalized state.
+   */
+  CLUSTER_NEEDS_FINALIZATION(r -> true,
+      ctx -> ctx.versionManager().needsFinalization()),
+  /**
+   * Classifies validations that has to run, when the client uses an older
+   * protocol version than the server.
+   */
+  OLDER_CLIENT_REQUESTS(r -> r.getVersion() < ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run, when the client uses a newer
+   * protocol version than the server.
+   */
+  NEWER_CLIENT_REQUESTS(r -> r.getVersion() > ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run for every request.
+   * If you plan to use this, please justify why the validation code should not
+   * be part of the actual request handling code.
+   */
+  UNCONDITIONAL(r -> true, ctx -> true);
+
+  private final Predicate<OMRequest> shouldApplyTo;
+  private final Predicate<ValidationContext> shouldApplyIn;
+
+  ValidationCondition(
+      Predicate<OMRequest> shouldApplyTo,
+      Predicate<ValidationContext> shouldApplyIn) {
+    this.shouldApplyTo = shouldApplyTo;
+    this.shouldApplyIn = shouldApplyIn;
+  }
+
+  boolean shouldApply(OMRequest req, ValidationContext ctx) {
+    return shouldApplyTo.test(req) && shouldApplyIn.test(ctx);

Review comment:
       Since we are just ANDing the conditions together, would it be easier to have ValidationConditions take just one predicate, and let the predicate's implementation define what conditions to act on. The current implementation seems to frequently require these `true` placeholders. I think it would read clearer if ValidationConditions were declared like:
   ```java
     CLUSTER_NEEDS_FINALIZATION(ctx -> ctx.versionManager().needsFinalization()),
     OLDER_CLIENT_REQUESTS(r -> r.getVersion() < ClientVersions.CURRENT_VERSION),
     NEWER_CLIENT_REQUESTS(r -> r.getVersion() > ClientVersions.CURRENT_VERSION),
     UNCONDITIONAL(r -> true);
   ```

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidatorRegistry.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.reflections.Reflections;
+import org.reflections.scanners.MethodAnnotationsScanner;
+import org.reflections.util.ClasspathHelper;
+import org.reflections.util.ConfigurationBuilder;
+
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Registry that loads and stores the request validators to be applied by
+ * a service.
+ */
+public class ValidatorRegistry {
+
+  private final EnumMap<
+      ValidationCondition, EnumMap<Type, Pair<List<Method>, List<Method>>>>
+      validators = new EnumMap<>(ValidationCondition.class);

Review comment:
       It looks like the pair in this structure is being used to determine the processing phase, with left meaning pre and right meaning post. However it looks like each method will only ever be a pre or post processor, so only one half of the pair is used for each entry.
   
   My understanding of the usage of this method is that given a ValidationCondition, RequestType, and RequestProcessingPhase we can get a list of corresponding methods to do the validation. I think `ValidationCondition, EnumMap<Type, EnumMap<RequestProcessingPhase, List<Method>>>>` would be a little more intuitive.

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/testvalidatorset1/GeneralValidatorsForTesting.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation.testvalidatorset1;
+
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.TestRequestValidations;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase.POST_PROCESS;
+import static org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase.PRE_PROCESS;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.CLUSTER_NEEDS_FINALIZATION;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.NEWER_CLIENT_REQUESTS;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.OLDER_CLIENT_REQUESTS;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.UNCONDITIONAL;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateKey;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateVolume;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.DeleteKeys;
+
+/**
+ * Some annotated request validator method, and facilities to help check if
+ * validations were properly called from tests where applicable.
+ */
+public final class GeneralValidatorsForTesting {
+
+  /**
+   * As the validators written here does not override any request or response
+   * but throw exceptions for specific tests, a test that wants to directly
+   * use a validator here, has to turn on this boolean, and the method that
+   * the test relies on has to check for this value.
+   *
+   * This is necessary to do not affect other tests that are testing requests
+   * processing, as for some of those tests this package is on the classpath
+   * and therefore the annotated validations are loadede for them.
+   */
+  private static boolean validatorTestsRunning = false;
+
+  public static void startValidatorTest() {
+    validatorTestsRunning = true;
+  }
+
+  public static void finishValidatorTest() {
+    validatorTestsRunning = false;
+  }
+
+  private GeneralValidatorsForTesting() { }
+
+  /**
+   * Interface to easily add listeners that get notified if a certain validator
+   * method defined in this class was called.
+   *
+   * @see TestRequestValidations for more details on how this intercace is
+   *      being used.
+   */
+  @FunctionalInterface
+  public interface ValidationListener {
+    void validationCalled(String calledMethodName);
+  }
+
+  private static List<ValidationListener> listeners = new ArrayList<>();
+
+  public static void addListener(ValidationListener listener) {
+    listeners.add(listener);
+  }
+
+  public static void removeListener(ValidationListener listener) {
+    listeners.remove(listener);
+  }
+
+  private static void fireValidationEvent(String calledMethodName) {
+    listeners.forEach(l -> l.validationCalled(calledMethodName));
+  }
+
+  @RequestFeatureValidator(
+      conditions = { CLUSTER_NEEDS_FINALIZATION },
+      processingPhase = PRE_PROCESS,
+      requestType = CreateKey)
+  public static OMRequest preFinalizePreProcessCreateKeyValidator(
+      OMRequest req, ValidationContext ctx) {
+    fireValidationEvent("preFinalizePreProcessCreateKeyValidator");
+    return req;
+  }
+
+  @RequestFeatureValidator(
+      conditions = { CLUSTER_NEEDS_FINALIZATION },
+      processingPhase = POST_PROCESS,
+      requestType = CreateKey)
+  public static OMResponse preFinalizePostProcessCreateKeyValidator(

Review comment:
       Annotating the validator instead of the request isn't bad, but would it be possible to rework it so the annotation goes on the request, and the validator is passed in as a functional interface to the annotation? Right now request annotations like `@DisallowedUntilLayoutVersion` go on the request itself, so if this set of annotations is not on the request, it may be difficult to figure out what processing each request has. Even if I know where to look as an upgrade dev, I think annotating the request would be more obvious/intuitive for the rest of the community who may need to use this feature.




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
fapifta commented on pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#issuecomment-1071880356


   One of the failing flaky tests is TestOzoneManagerHAMetadataOnly which I tried to run locally, and it ran green, while the other two was ignored earlier, and does not seem to work at all now.
   
   Based on this and the +1 from @errose28 I am merging this one to master.
   Thank you for the reviews!


-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kerneltime commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
kerneltime commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r809362752



##########
File path: dev-support/annotations/pom.xml
##########
@@ -0,0 +1,92 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+https://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <groupId>org.apache.ozone</groupId>
+  <artifactId>ozone-annotation-processing</artifactId>
+  <version>1.3.0-SNAPSHOT</version>
+  <description>Apache Ozone annotation processing tools for validating custom
+    annotations at compile time.
+  </description>
+  <name>Apache Ozone Annotation Processing</name>
+  <packaging>jar</packaging>
+
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    <spotbugs.version>3.1.12</spotbugs.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.github.spotbugs</groupId>
+      <artifactId>spotbugs</artifactId>
+      <version>${spotbugs.version}</version>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <!--
+       | It is required to add spotbugs dependency here, so that CI can check
+       | this package properly and does not fail the findbugs check.
+      -->
+      <plugin>
+        <groupId>com.github.spotbugs</groupId>
+        <artifactId>spotbugs-maven-plugin</artifactId>
+        <version>${spotbugs.version}</version>
+        <configuration>
+          <maxHeap>1024</maxHeap>
+          <xmlOutput>true</xmlOutput>
+          <includeTests>true</includeTests>
+        </configuration>
+      </plugin>
+      <!--
+       | Compiler plugin configuration have to be modified to compile the
+       | annotation processor, without having any annotation processor applied
+       | as due to the presence of
+       | META-INF/services/javac.annotation.processing.Processor the compilation
+       | would fail as the linked processor class can not be find while we are
+       | compiling it.
+      -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>3.9.0</version>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+        </configuration>
+        <executions>
+          <execution>
+            <id>default-compile</id>
+            <configuration>
+              <compilerArgument>-proc:none</compilerArgument>

Review comment:
       Please add a note as to why we need `-proc:none` and the sequence of events that should occur when we fresh build the repo.




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on a change in pull request #3104: Request validation

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r808545566



##########
File path: hadoop-hdds/interface-client/pom.xml
##########
@@ -47,6 +47,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>javax.annotation</groupId>
       <artifactId>javax.annotation-api</artifactId>
     </dependency>
+    <dependency>

Review comment:
       I know this one is weird, but without this, the hdds-interface-client module was not compiling, and it was missing a class from this package once I have added our annotation-processing module. I would be glad if someone can explain what is the reason behind the necessity of this dependency definition after adding the ozone-annotation-processing module.




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta merged pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
fapifta merged pull request #3104:
URL: https://github.com/apache/ozone/pull/3104


   


-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r819985366



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidationCondition.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+
+import java.util.function.Predicate;
+
+/**
+ * Defines conditions for which validators can be assigned to.
+ *
+ * These conditions describe a situation where special request handling might
+ * be necessary. In these cases we do not override the actual request handling
+ * code, but based on certain request properties we might reject a request
+ * early, or we might modify the request, or the response received/sent from/to
+ * the client.
+ */
+public enum ValidationCondition {
+  /**
+   * Classifies validations that has to run after an upgrade until the cluster
+   * is in a pre-finalized state.
+   */
+  CLUSTER_NEEDS_FINALIZATION(r -> true,
+      ctx -> ctx.versionManager().needsFinalization()),
+  /**
+   * Classifies validations that has to run, when the client uses an older
+   * protocol version than the server.
+   */
+  OLDER_CLIENT_REQUESTS(r -> r.getVersion() < ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run, when the client uses a newer
+   * protocol version than the server.
+   */
+  NEWER_CLIENT_REQUESTS(r -> r.getVersion() > ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run for every request.
+   * If you plan to use this, please justify why the validation code should not
+   * be part of the actual request handling code.
+   */
+  UNCONDITIONAL(r -> true, ctx -> true);

Review comment:
       This one is kind of a dilemma for me as well, I have removed these two for now, however I started to look into the possibilities of extending the framework to these use cases on the client side, and what changes might be necessary to use a similar approach there and whether it would be beneficial or not... But in case it will happen we can re-introduce the NEWER_CLIENT_REQUESTS, once we start to think about moving out the authorization or audit for example, we can re-introduce UNCONDITIONAL, and if we change both things, we might forget about them :)
   
   Just to do not loose the tests for these enum values, I have just commented out test code, and the enum values, so we can reintroduce them easier if we want.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidationCondition.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+
+import java.util.function.Predicate;
+
+/**
+ * Defines conditions for which validators can be assigned to.
+ *
+ * These conditions describe a situation where special request handling might
+ * be necessary. In these cases we do not override the actual request handling
+ * code, but based on certain request properties we might reject a request
+ * early, or we might modify the request, or the response received/sent from/to
+ * the client.
+ */
+public enum ValidationCondition {
+  /**
+   * Classifies validations that has to run after an upgrade until the cluster
+   * is in a pre-finalized state.
+   */
+  CLUSTER_NEEDS_FINALIZATION(r -> true,
+      ctx -> ctx.versionManager().needsFinalization()),
+  /**
+   * Classifies validations that has to run, when the client uses an older
+   * protocol version than the server.
+   */
+  OLDER_CLIENT_REQUESTS(r -> r.getVersion() < ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run, when the client uses a newer
+   * protocol version than the server.
+   */
+  NEWER_CLIENT_REQUESTS(r -> r.getVersion() > ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run for every request.
+   * If you plan to use this, please justify why the validation code should not
+   * be part of the actual request handling code.
+   */
+  UNCONDITIONAL(r -> true, ctx -> true);
+
+  private final Predicate<OMRequest> shouldApplyTo;
+  private final Predicate<ValidationContext> shouldApplyIn;
+
+  ValidationCondition(
+      Predicate<OMRequest> shouldApplyTo,
+      Predicate<ValidationContext> shouldApplyIn) {
+    this.shouldApplyTo = shouldApplyTo;
+    this.shouldApplyIn = shouldApplyIn;
+  }
+
+  boolean shouldApply(OMRequest req, ValidationContext ctx) {
+    return shouldApplyTo.test(req) && shouldApplyIn.test(ctx);

Review comment:
       As predicates are type parametrized, we can not use just one predicate that processes either a context or a request, but you are right, it would be way more better to have a more simpler approach to express the condition.
   
   How about using an abstract method and overriding it on a per enum value basis like the one I added instead of the old way?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidatorRegistry.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.reflections.Reflections;
+import org.reflections.scanners.MethodAnnotationsScanner;
+import org.reflections.util.ClasspathHelper;
+import org.reflections.util.ConfigurationBuilder;
+
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Registry that loads and stores the request validators to be applied by
+ * a service.
+ */
+public class ValidatorRegistry {
+
+  private final EnumMap<
+      ValidationCondition, EnumMap<Type, Pair<List<Method>, List<Method>>>>
+      validators = new EnumMap<>(ValidationCondition.class);

Review comment:
       For a condition and a request type there can be pre and post methods also defined, and if so, then both sides of the pair is used. However your understanding is correct, and the approach you are proposing is also way more forward looking as even though we have just two processing phase right now, the structure you are proposing covers any additional processing phase, and does not limit our ability to add a new phase at any time by extending the enum.
   
   I have changed the implementation to use enummap of enummap of enummap instead of enummap of enummap of pairs.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidatorRegistry.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.reflections.Reflections;
+import org.reflections.scanners.MethodAnnotationsScanner;
+import org.reflections.util.ClasspathHelper;
+import org.reflections.util.ConfigurationBuilder;
+
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Registry that loads and stores the request validators to be applied by
+ * a service.
+ */
+public class ValidatorRegistry {
+
+  private final EnumMap<
+      ValidationCondition, EnumMap<Type, Pair<List<Method>, List<Method>>>>
+      validators = new EnumMap<>(ValidationCondition.class);
+
+  /**
+   * Creates a {@link ValidatorRegistry} instance that discovers validation
+   * methods in the provided package and the packages in the same resource.
+   * A validation method is recognized by the {@link RequestFeatureValidator}
+   * annotation that contains important information about how and when to use
+   * the validator.
+   * @param validatorPackage the main package inside which validatiors should
+   *                         be discovered.
+   */
+  ValidatorRegistry(String validatorPackage) {
+    this(ClasspathHelper.forPackage(validatorPackage));
+  }
+
+  /**
+   * Creates a {@link ValidatorRegistry} instance that discovers validation
+   * methods under the provided URL.
+   * A validation method is recognized by the {@link RequestFeatureValidator}
+   * annotation that contains important information about how and when to use
+   * the validator.
+   * @param searchUrls the path in which the annotated methods are searched.
+   */
+  ValidatorRegistry(Collection<URL> searchUrls) {
+    Reflections reflections = new Reflections(new ConfigurationBuilder()
+        .setUrls(searchUrls)
+        .setScanners(new MethodAnnotationsScanner())
+        .useParallelExecutor()
+    );
+
+    Set<Method> describedValidators =
+        reflections.getMethodsAnnotatedWith(RequestFeatureValidator.class);
+    initMaps(describedValidators);
+  }
+
+  /**
+   * Get the validators that has to be run in the given list of
+   * {@link ValidationCondition}s, for the given requestType and
+   * {@link RequestProcessingPhase}.
+   *
+   * @param conditions conditions that are present for the request
+   * @param requestType the type of the protocol message
+   * @param phase the request processing phase
+   * @return the list of validation methods that has to run.
+   */
+  List<Method> validationsFor(
+      List<ValidationCondition> conditions,
+      Type requestType,
+      RequestProcessingPhase phase) {
+
+    if (conditions.isEmpty() || validators.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    Set<Method> returnValue =
+        new HashSet<>(validationsFor(conditions.get(0), requestType, phase));

Review comment:
       You can define the same validator to be used for multiple conditions, as the conditions field in the annotation is defined to be an array. In this case the validator will be added into the list inside both condition for the request type and processing phase.
   
   Also when you are requesting for the validation via this method, you can specify multiple conditions, as the cluster can be in a pre-finalized state when an older client issues a request, and in this case if the same validation is set for both conditions, it would be returned twice, and then the validator would run twice which might be either unnecessary, unexpected or misbehaving the second time.
   
   As you might ask for a use case, I do not have one in mind, but having this in does not seem to be a big price for having this flexibility, though if you think, I am open to drop this, and we can change the conditions to be condition, and let just one condition to be specified in the annotation.

##########
File path: dev-support/annotations/src/main/java/org/apache/ozone/annotations/RequestFeatureValidatorProcessor.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.annotations;
+
+import javax.annotation.processing.AbstractProcessor;
+import javax.annotation.processing.RoundEnvironment;
+import javax.annotation.processing.SupportedAnnotationTypes;
+import javax.annotation.processing.SupportedSourceVersion;
+import javax.lang.model.SourceVersion;
+import javax.lang.model.element.AnnotationMirror;
+import javax.lang.model.element.AnnotationValue;
+import javax.lang.model.element.AnnotationValueVisitor;
+import javax.lang.model.element.Element;
+import javax.lang.model.element.ElementKind;
+import javax.lang.model.element.ExecutableElement;
+import javax.lang.model.element.Modifier;
+import javax.lang.model.element.TypeElement;
+import javax.lang.model.element.VariableElement;
+import javax.lang.model.type.ExecutableType;
+import javax.lang.model.type.TypeMirror;
+import javax.lang.model.util.SimpleAnnotationValueVisitor8;
+import javax.tools.Diagnostic;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+
+@SupportedAnnotationTypes(
+    "org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator")
+@SupportedSourceVersion(SourceVersion.RELEASE_8)
+public class RequestFeatureValidatorProcessor extends AbstractProcessor {

Review comment:
       Done, thank you for noting this, altough this indicatest that checkstyle is not running for this module, as the relevant checkstyle rule should have been failed for not having javadoc for a class...
   I have looked into this, and added checkstyle checks to the module, also fixed the checkstyle issues discovered.

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/testvalidatorset1/GeneralValidatorsForTesting.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation.testvalidatorset1;
+
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.TestRequestValidations;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase.POST_PROCESS;
+import static org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase.PRE_PROCESS;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.CLUSTER_NEEDS_FINALIZATION;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.NEWER_CLIENT_REQUESTS;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.OLDER_CLIENT_REQUESTS;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.UNCONDITIONAL;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateKey;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateVolume;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.DeleteKeys;
+
+/**
+ * Some annotated request validator method, and facilities to help check if
+ * validations were properly called from tests where applicable.
+ */
+public final class GeneralValidatorsForTesting {
+
+  /**
+   * As the validators written here does not override any request or response
+   * but throw exceptions for specific tests, a test that wants to directly
+   * use a validator here, has to turn on this boolean, and the method that
+   * the test relies on has to check for this value.
+   *
+   * This is necessary to do not affect other tests that are testing requests
+   * processing, as for some of those tests this package is on the classpath
+   * and therefore the annotated validations are loadede for them.
+   */
+  private static boolean validatorTestsRunning = false;
+
+  public static void startValidatorTest() {
+    validatorTestsRunning = true;
+  }
+
+  public static void finishValidatorTest() {
+    validatorTestsRunning = false;
+  }
+
+  private GeneralValidatorsForTesting() { }
+
+  /**
+   * Interface to easily add listeners that get notified if a certain validator
+   * method defined in this class was called.
+   *
+   * @see TestRequestValidations for more details on how this intercace is
+   *      being used.
+   */
+  @FunctionalInterface
+  public interface ValidationListener {
+    void validationCalled(String calledMethodName);
+  }
+
+  private static List<ValidationListener> listeners = new ArrayList<>();
+
+  public static void addListener(ValidationListener listener) {
+    listeners.add(listener);
+  }
+
+  public static void removeListener(ValidationListener listener) {
+    listeners.remove(listener);
+  }
+
+  private static void fireValidationEvent(String calledMethodName) {
+    listeners.forEach(l -> l.validationCalled(calledMethodName));
+  }
+
+  @RequestFeatureValidator(
+      conditions = { CLUSTER_NEEDS_FINALIZATION },
+      processingPhase = PRE_PROCESS,
+      requestType = CreateKey)
+  public static OMRequest preFinalizePreProcessCreateKeyValidator(
+      OMRequest req, ValidationContext ctx) {
+    fireValidationEvent("preFinalizePreProcessCreateKeyValidator");
+    return req;
+  }
+
+  @RequestFeatureValidator(
+      conditions = { CLUSTER_NEEDS_FINALIZATION },
+      processingPhase = POST_PROCESS,
+      requestType = CreateKey)
+  public static OMResponse preFinalizePostProcessCreateKeyValidator(

Review comment:
       As we discussed in one of our meetings, this is an interesting suggestion, and I haven't put too much though in using the annotation on the requests itself before.
   
   If we have the annotation on a request class, then the request type is defined by the request class, so we can skip that, still we remain with a set of methods for four different cases: prefinalize-preprocess, oldclient-preprocess, prefinalize-postprocess, oldclient-postprocess. These methods have to be declared to be used within the annotation, or somewhere. If we want to introduce a new condition or a new processing phase, we would need to introduce more lists for the new cases either in the annotation or elsewhere. An other option might be a naming convention for the methods, but that is a bit harder to check at compile time, and may be error prone.
   
   Pre and post differentiation can not be skipped as the methods for the two has to have a different signature, so it can not fit within the same functional interface. Conditions I believe has to be separated, if not, that would introduce branching in the code of the validator later when any new validation functionality is added.
   An alternative for branching when a new validator is added to add multiple validations to a certain case, but in that case the annotation gets bigger and bigger so it becomes harder to see what is happening there.
   
   
   I know we have mentioned the two phase write request processing as a potential blocker, but all in all that does not seem to be a relevant difference from the point where this framework is integrated to the request processing flow, still I would say it is more beneficial to have small separate methods that do just one thing to be defined as a validator separately, instead of trying to collect all validators in one place in one/multiple annotations before the class declaration, or leaving it to be a potentially growing single method with growing number of code branches, but if you see a potential way of solving this without the mentioned risks, I am open to discuss, I don't see that this change makes life easier or simpler on the long run unfortunately.




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kerneltime commented on pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
kerneltime commented on pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#issuecomment-1042468628


   @errose28 this is a more generic framework for adding request pre and post validations. The fields in place for OM and Client are the same this is an alternative way to define and invoke validations. 


-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kerneltime commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
kerneltime commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r809460255



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation2/ValidatorsForOnlyNewClientValidations.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation2;

Review comment:
       Is this package for a sample? Not sure why it needs to be there.




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta merged pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
fapifta merged pull request #3104:
URL: https://github.com/apache/ozone/pull/3104


   


-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r809564663



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation2/ValidatorsForOnlyNewClientValidations.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation2;

Review comment:
       This one is here for a test case.
   The bulk of the tests are using the simple validation package, but in that we have validators for all kinds of stuff.
   This package is created so that when the validator registry is instantiated, we can provide this package to scan for some of the tests, and we can test for cases when there are no validators for specific conditions, or phases or request types. As the discovery discovers classes with an annotation under a classpath url, I can not test for both, hence I created this package where I have just one specific validation.




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 commented on pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
errose28 commented on pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#issuecomment-1042459275


   Thanks for creating this @fapifta. I haven't had a chance to look it over yet, but I'm just wondering how this relates to the client versioning in https://github.com/apache/ozone/pull/3031? Is this a replacement for that PR or is it designed to work together with it?


-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
errose28 commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r817127615



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidationCondition.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+
+import java.util.function.Predicate;
+
+/**
+ * Defines conditions for which validators can be assigned to.
+ *
+ * These conditions describe a situation where special request handling might
+ * be necessary. In these cases we do not override the actual request handling
+ * code, but based on certain request properties we might reject a request
+ * early, or we might modify the request, or the response received/sent from/to
+ * the client.
+ */
+public enum ValidationCondition {
+  /**
+   * Classifies validations that has to run after an upgrade until the cluster
+   * is in a pre-finalized state.
+   */
+  CLUSTER_NEEDS_FINALIZATION(r -> true,
+      ctx -> ctx.versionManager().needsFinalization()),
+  /**
+   * Classifies validations that has to run, when the client uses an older
+   * protocol version than the server.
+   */
+  OLDER_CLIENT_REQUESTS(r -> r.getVersion() < ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run, when the client uses a newer
+   * protocol version than the server.
+   */
+  NEWER_CLIENT_REQUESTS(r -> r.getVersion() > ClientVersions.CURRENT_VERSION,
+      ctx -> true),
+  /**
+   * Classifies validations that has to run for every request.
+   * If you plan to use this, please justify why the validation code should not
+   * be part of the actual request handling code.
+   */
+  UNCONDITIONAL(r -> true, ctx -> true);
+
+  private final Predicate<OMRequest> shouldApplyTo;
+  private final Predicate<ValidationContext> shouldApplyIn;
+
+  ValidationCondition(
+      Predicate<OMRequest> shouldApplyTo,
+      Predicate<ValidationContext> shouldApplyIn) {
+    this.shouldApplyTo = shouldApplyTo;
+    this.shouldApplyIn = shouldApplyIn;
+  }
+
+  boolean shouldApply(OMRequest req, ValidationContext ctx) {
+    return shouldApplyTo.test(req) && shouldApplyIn.test(ctx);

Review comment:
       Since we are just ANDing the conditions together, would it be easier to have ValidationConditions take just one predicate, and let the predicate's implementation define what conditions to act on? The current implementation seems to frequently require these `true` placeholders. I think it would read clearer if ValidationConditions were declared like:
   ```java
     CLUSTER_NEEDS_FINALIZATION(ctx -> ctx.versionManager().needsFinalization()),
     OLDER_CLIENT_REQUESTS(r -> r.getVersion() < ClientVersions.CURRENT_VERSION),
     NEWER_CLIENT_REQUESTS(r -> r.getVersion() > ClientVersions.CURRENT_VERSION),
     UNCONDITIONAL(r -> true);
   ```




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
fapifta commented on pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#issuecomment-1071880356


   One of the failing flaky tests is TestOzoneManagerHAMetadataOnly which I tried to run locally, and it ran green, while the other two was ignored earlier, and does not seem to work at all now.
   
   Based on this and the +1 from @errose28 I am merging this one to master.
   Thank you for the reviews!


-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
errose28 commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r824973201



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
##########
@@ -0,0 +1,422 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.ValidationListener;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.ozone.upgrade.LayoutVersionManager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.request.validation.ValidationContext.of;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.startValidatorTest;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.finishValidatorTest;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status.OK;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateKey;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.DeleteKeys;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.RenameKey;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Testing the RequestValidations class that is used to run the validation for
+ * any given request that arrives to OzoneManager.
+ */
+public class TestRequestValidations {
+  private static final String PACKAGE =
+      "org.apache.hadoop.ozone.om.request.validation.testvalidatorset1";
+
+  private static final String PACKAGE_WO_VALIDATORS =
+      "org.apache.hadoop.hdds.annotation";
+
+  private final ValidationListenerImpl validationListener =
+      new ValidationListenerImpl();
+
+  @Before
+  public void setup() {
+    startValidatorTest();
+    validationListener.attach();
+  }
+
+  @After
+  public void tearDown() {
+    validationListener.detach();
+    finishValidatorTest();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutLoading() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .withinContext(of(aFinalizedVersionManager()))
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutContext() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test
+  public void testUsingRegistryWithoutPackage() throws ServiceException {
+    new RequestValidations()
+        .withinContext(of(aFinalizedVersionManager()))
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+//    validationListener
+//        .assertCalled("unconditionalPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testNoPreValdiationsWithoutValidationMethods()
+      throws ServiceException {
+    int omVersion = 0;
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadEmptyValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(omVersion));
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPostValdiationsWithoutValidationMethods()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadEmptyValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(currentClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPreValidationsRunningForRequestTypeWithoutValidators()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aRenameKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPostValidationsAreRunningForRequestTypeWithoutValidators()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aRenameKeyRequest(currentClientVersion()), aRenameKeyResponse());
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+/*  @Test

Review comment:
       Can we add a comment explaining why these are commented out (the corresponding validators are commented out since they are not needed)? Same for the below commented out tests.

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/testvalidatorset1/GeneralValidatorsForTesting.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation.testvalidatorset1;
+
+import org.apache.hadoop.ozone.om.request.validation.RequestFeatureValidator;
+import org.apache.hadoop.ozone.om.request.validation.TestRequestValidations;
+import org.apache.hadoop.ozone.om.request.validation.ValidationContext;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase.POST_PROCESS;
+import static org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase.PRE_PROCESS;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.CLUSTER_NEEDS_FINALIZATION;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.NEWER_CLIENT_REQUESTS;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.OLDER_CLIENT_REQUESTS;
+import static org.apache.hadoop.ozone.om.request.validation.ValidationCondition.UNCONDITIONAL;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateKey;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateVolume;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.DeleteKeys;
+
+/**
+ * Some annotated request validator method, and facilities to help check if
+ * validations were properly called from tests where applicable.
+ */
+public final class GeneralValidatorsForTesting {
+
+  /**
+   * As the validators written here does not override any request or response
+   * but throw exceptions for specific tests, a test that wants to directly
+   * use a validator here, has to turn on this boolean, and the method that
+   * the test relies on has to check for this value.
+   *
+   * This is necessary to do not affect other tests that are testing requests
+   * processing, as for some of those tests this package is on the classpath
+   * and therefore the annotated validations are loadede for them.
+   */
+  private static boolean validatorTestsRunning = false;
+
+  public static void startValidatorTest() {
+    validatorTestsRunning = true;
+  }
+
+  public static void finishValidatorTest() {
+    validatorTestsRunning = false;
+  }
+
+  private GeneralValidatorsForTesting() { }
+
+  /**
+   * Interface to easily add listeners that get notified if a certain validator
+   * method defined in this class was called.
+   *
+   * @see TestRequestValidations for more details on how this intercace is
+   *      being used.
+   */
+  @FunctionalInterface
+  public interface ValidationListener {
+    void validationCalled(String calledMethodName);
+  }
+
+  private static List<ValidationListener> listeners = new ArrayList<>();
+
+  public static void addListener(ValidationListener listener) {
+    listeners.add(listener);
+  }
+
+  public static void removeListener(ValidationListener listener) {
+    listeners.remove(listener);
+  }
+
+  private static void fireValidationEvent(String calledMethodName) {
+    listeners.forEach(l -> l.validationCalled(calledMethodName));
+  }
+
+  @RequestFeatureValidator(
+      conditions = { CLUSTER_NEEDS_FINALIZATION },
+      processingPhase = PRE_PROCESS,
+      requestType = CreateKey)
+  public static OMRequest preFinalizePreProcessCreateKeyValidator(
+      OMRequest req, ValidationContext ctx) {
+    fireValidationEvent("preFinalizePreProcessCreateKeyValidator");
+    return req;
+  }
+
+  @RequestFeatureValidator(
+      conditions = { CLUSTER_NEEDS_FINALIZATION },
+      processingPhase = POST_PROCESS,
+      requestType = CreateKey)
+  public static OMResponse preFinalizePostProcessCreateKeyValidator(

Review comment:
       Thanks for the analysis. I agree coming up with a way to do this would complicate the implementation. I am ok with the current approach.

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
##########
@@ -0,0 +1,422 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.ValidationListener;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.ozone.upgrade.LayoutVersionManager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.request.validation.ValidationContext.of;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.startValidatorTest;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.finishValidatorTest;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status.OK;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateKey;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.DeleteKeys;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.RenameKey;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Testing the RequestValidations class that is used to run the validation for
+ * any given request that arrives to OzoneManager.
+ */
+public class TestRequestValidations {
+  private static final String PACKAGE =
+      "org.apache.hadoop.ozone.om.request.validation.testvalidatorset1";
+
+  private static final String PACKAGE_WO_VALIDATORS =
+      "org.apache.hadoop.hdds.annotation";
+
+  private final ValidationListenerImpl validationListener =
+      new ValidationListenerImpl();
+
+  @Before
+  public void setup() {
+    startValidatorTest();
+    validationListener.attach();
+  }
+
+  @After
+  public void tearDown() {
+    validationListener.detach();
+    finishValidatorTest();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutLoading() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .withinContext(of(aFinalizedVersionManager()))
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutContext() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test
+  public void testUsingRegistryWithoutPackage() throws ServiceException {
+    new RequestValidations()
+        .withinContext(of(aFinalizedVersionManager()))
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+//    validationListener
+//        .assertCalled("unconditionalPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testNoPreValdiationsWithoutValidationMethods()
+      throws ServiceException {
+    int omVersion = 0;
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadEmptyValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(omVersion));
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPostValdiationsWithoutValidationMethods()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadEmptyValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(currentClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPreValidationsRunningForRequestTypeWithoutValidators()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aRenameKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPostValidationsAreRunningForRequestTypeWithoutValidators()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aRenameKeyRequest(currentClientVersion()), aRenameKeyResponse());
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+/*  @Test
+  public void testUnconditionalPreProcessValidationsAreCalled()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(1);
+    validationListener
+        .assertCalled("unconditionalPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testUnconditionalPostProcessValidationsAreCalled()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(currentClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(1);
+    validationListener
+        .assertCalled("unconditionalPostProcessCreateKeyValidator");
+  }*/
+
+  @Test
+  public void testPreProcessorExceptionHandling() {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    try {
+      validations.validateRequest(aDeleteKeysRequest(olderClientVersion()));
+      fail("ServiceException was expected but was not thrown.");
+    } catch (ServiceException ignored) { }
+
+    validationListener.assertNumOfEvents(1);
+    validationListener.assertCalled("throwingPreProcessValidator");
+  }
+
+  @Test
+  public void testPostProcessorExceptionHandling() {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    try {
+      validations.validateResponse(
+          aDeleteKeysRequest(olderClientVersion()), aDeleteKeysResponse());
+      fail("ServiceException was expected but was not thrown.");
+    } catch (ServiceException ignored) { }
+
+    validationListener.assertNumOfEvents(1);
+    validationListener.assertCalled("throwingPostProcessValidator");
+  }
+
+/*  @Test
+  public void testNewClientConditionIsRecognizedAndPreValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(newerClientVersion()));
+
+    validationListener.assertNumOfEvents(2);
+    validationListener.assertAllCalled(
+        "unconditionalPreProcessCreateKeyValidator",
+        "newClientPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testNewClientConditionIsRecognizedAndPostValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(newerClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(3);
+    validationListener.assertAllCalled(
+        "unconditionalPostProcessCreateKeyValidator",
+        "newClientPostProcessCreateKeyValidator",
+        "newClientPostProcessCreateKeyValidator2");
+  }*/
+
+  @Test
+  public void testOldClientConditionIsRecognizedAndPreValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(olderClientVersion()));
+
+    validationListener.assertNumOfEvents(1);
+    validationListener.assertAllCalled(
+//        "unconditionalPreProcessCreateKeyValidator",
+        "oldClientPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testOldClientConditionIsRecognizedAndPostValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(olderClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(2);
+    validationListener.assertAllCalled(
+//        "unconditionalPostProcessCreateKeyValidator",
+        "oldClientPostProcessCreateKeyValidator",
+        "oldClientPostProcessCreateKeyValidator2");
+  }
+
+  @Test
+  public void testPreFinalizedWithOldClientConditionPreProcValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(anUnfinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(olderClientVersion()));
+
+    validationListener.assertNumOfEvents(2);
+    validationListener.assertAllCalled(
+//        "unconditionalPreProcessCreateKeyValidator",
+        "preFinalizePreProcessCreateKeyValidator",
+        "oldClientPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testPreFinalizedWithOldClientConditionPostProcValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(anUnfinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(olderClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(3);
+    validationListener.assertAllCalled(
+//        "unconditionalPostProcessCreateKeyValidator",
+        "preFinalizePostProcessCreateKeyValidator",
+        "oldClientPostProcessCreateKeyValidator",
+        "oldClientPostProcessCreateKeyValidator2");
+  }
+
+  private RequestValidations loadValidations(ValidationContext ctx) {
+    return new RequestValidations()
+        .fromPackage(PACKAGE)
+        .withinContext(ctx)
+        .load();
+  }
+
+  private RequestValidations loadEmptyValidations(ValidationContext ctx) {
+    return new RequestValidations()
+        .fromPackage(PACKAGE_WO_VALIDATORS)
+        .withinContext(ctx)
+        .load();
+  }
+
+  private int olderClientVersion() {
+    return ClientVersions.CURRENT_VERSION - 1;
+  }
+
+  private int currentClientVersion() {
+    return ClientVersions.CURRENT_VERSION;
+  }
+
+/*  private int newerClientVersion() {
+    return ClientVersions.CURRENT_VERSION + 1;
+  }*/
+
+  private OMRequest aCreateKeyRequest(int clientVersion) {
+    return aRequest(CreateKey, clientVersion);
+  }
+
+  private OMRequest aDeleteKeysRequest(int clientVersion) {
+    return aRequest(DeleteKeys, clientVersion);
+  }
+
+  private OMRequest aRenameKeyRequest(int clientVersion) {
+    return aRequest(RenameKey, clientVersion);
+  }
+
+  private OMRequest aRequest(Type type, int clientVersion) {
+    return OMRequest.newBuilder()
+        .setVersion(clientVersion)
+        .setCmdType(type)
+        .setClientId("TestClient")
+        .build();
+  }
+
+  private OMResponse aCreateKeyResponse() {
+    return aResponse(CreateKey);
+  }
+
+  private OMResponse aDeleteKeysResponse() {
+    return aResponse(DeleteKeys);
+  }
+
+  private OMResponse aRenameKeyResponse() {
+    return aResponse(RenameKey);
+  }
+
+  private OMResponse aResponse(Type type) {
+    return OMResponse.newBuilder()
+        .setCmdType(type)
+        .setStatus(OK)
+        .build();
+  }
+
+  private LayoutVersionManager aFinalizedVersionManager() {
+    LayoutVersionManager vm = mock(LayoutVersionManager.class);
+    when(vm.needsFinalization()).thenReturn(false);
+    return vm;
+  }
+
+  private LayoutVersionManager anUnfinalizedVersionManager() {
+    LayoutVersionManager vm = mock(LayoutVersionManager.class);
+    when(vm.needsFinalization()).thenReturn(true);
+    return vm;
+  }
+
+  private static class ValidationListenerImpl implements ValidationListener {
+    private List<String> calledMethods = new ArrayList<>();
+
+    @Override
+    public void validationCalled(String calledMethodName) {
+      calledMethods.add(calledMethodName);
+    }
+
+    public void attach() {
+      GeneralValidatorsForTesting.addListener(this);
+    }
+
+    public void detach() {
+      GeneralValidatorsForTesting.removeListener(this);
+      reset();
+    }
+
+    public void reset() {
+      calledMethods = new ArrayList<>();
+    }
+
+    public void assertCalled(String... methodNames) {

Review comment:
       Are there tests right now where this is used but `assertAllCalled` would not work in its place?

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
##########
@@ -0,0 +1,422 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.ValidationListener;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.ozone.upgrade.LayoutVersionManager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.request.validation.ValidationContext.of;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.startValidatorTest;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.finishValidatorTest;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status.OK;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateKey;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.DeleteKeys;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.RenameKey;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Testing the RequestValidations class that is used to run the validation for
+ * any given request that arrives to OzoneManager.
+ */
+public class TestRequestValidations {
+  private static final String PACKAGE =
+      "org.apache.hadoop.ozone.om.request.validation.testvalidatorset1";
+
+  private static final String PACKAGE_WO_VALIDATORS =
+      "org.apache.hadoop.hdds.annotation";
+
+  private final ValidationListenerImpl validationListener =
+      new ValidationListenerImpl();
+
+  @Before
+  public void setup() {
+    startValidatorTest();
+    validationListener.attach();
+  }
+
+  @After
+  public void tearDown() {
+    validationListener.detach();
+    finishValidatorTest();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutLoading() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .withinContext(of(aFinalizedVersionManager()))
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutContext() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test
+  public void testUsingRegistryWithoutPackage() throws ServiceException {
+    new RequestValidations()
+        .withinContext(of(aFinalizedVersionManager()))
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+//    validationListener
+//        .assertCalled("unconditionalPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testNoPreValdiationsWithoutValidationMethods()

Review comment:
       nit. typo




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kerneltime commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
kerneltime commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r809365843



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestFeatureValidator.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * An annotation to mark methods that does certain request validations.

Review comment:
       ```suggestion
    * An annotation to mark methods that do certain request validations.
   ```




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
fapifta commented on pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#issuecomment-1047315023


   Thank you @kerneltime for your notes, I have added answers for you.
   
   In the meantime, I was thinking about how to answer to @errose28 beyond the simple yes, to give some more context I have realized a few questions, for which I am trying to come up with good answers and reasoning, but still I feel we should go over them one by one, and discuss it once more. I am wondering what would be a good platform for that discussion.
   
   The questions:
   1. The condition NEW_CLIENT_REQUESTS, is something which is a bit futuristic, as how on earth an old server will new about a new client, and I was wondering if ever this will be relevant...
   2. I am still wondering if an unconditional validation will ever be a way we want to go under, as that has consequences on how we organize request handling code...
   3.  Ordering of the validators is again a good question, will there be ever an ordering required (if we use unconditionally hooked in methods, then certainly it will be required...)
   4. The current structure is also a good question, now we can define a validator in terms of the validation condition, the request type, and by the hook... Request type is tied to the protocol version, and it is really not easy now to add the same validation to two requests.
   
   To give some short answers:
   1. this we can use in rare cases, for example when an incompatible change went through, then this could be an easy way to provide a fix for those whom are having trouble because of that change within an already upgraded cluster and an old one in case those need to communicate from the newer code side. The cost of this is negligible when we look at maintenance or request processing performance.
   2. unconditionals can be used to move out authorization and auditing for example, but if we go down this route then naming of the classes and methods needs to change, as these are not strictly validations of request features...
   3. ordering is easy to add, and inevitable I believe if we implement those unconditionals mentioned in 2.
   4. is a good enforcer of a - I think - better design, as it forces an implementor to implement logic for one request only, and helps us avoid speculative generalism, with that switch-cases and ifs inside a validator method (how much of this is whisful thinking?), also gives a good rationale to put validators into the request processing classes. (Note: read requests' processing have to be moved out to their own classes for this though)
   
   
   The idea @kerneltime came up with to perhaps move some generic request processing part into validator methods, made me also think further...
   If we go with unconditionals and use them for such things like authorization of the request, then we probably should rename a few things:
   RequestFeatureValidator -> RequestPreProcessor and RequestPostProcessor (with that we can remove the RequestProcessingPhase enum)
   ValidatorRegistry -> RequestHookRegistry (or something like that)
   RequestValidations -> RequestHooks (or something like that)
   
   The idea to use this with other protocols than just the ozone client protocol, also brings us to some further changes that are necessary to extend the whole API:
   - move the whole thing to hdds-common
   - we might move the javac annotation processor to hdds/annotations
   - we need to generalize the request type in the annotations (also it would still be nice to validate that somehow, and that is hard to solve due to unknown dependencies, unless we have a generic annotation processor and specific annotation processors defined alongside the protocol implementation we want to hook into)
   - we still need one wrapper around the registry per protocol, as integration with the protocol might varies for different protocols, and request parameters might be different, this has to be examined
   - we have to decide where this would be useful, and we need to introduce client version into the protocols where it is missing.
   - probably more, I still just experimenting with this in my head... also I have no clue if there would be a necessary incompatible change down this route...
   
   Q1 also leads to one more thing, for which I did not do any research, so I just put it here as something we might examine later on: newer client requests condition can be something used on the client side, if there is such an easy way to hook things into the client side of the request, as in that case the client can use this condition to alter requests when talking to older servers... again... this came up just now in my mind, and I am absolutely unsure if we can easily hook on client side, and whether it makes any sense to do this there.
   (Note to self: the idea feels like if I found a golden hammer and I started to see nails everywhere, so please argue with the usefuleness of this...)
   
   Last but not least, there was a main concern from @kerneltime, how to ensure these methods will not linger somewhere in the code, and how we can be sure which methods were running during pre/post process phase for a given query. Also how we can make this easy for devs to realize that there are these validations.
   One answer is code organization (validators along with request handlers), and possibly an enforcement of that, an other is logging, these concerns are still not addressed in the code.
   


-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
errose28 commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r824323849



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/ValidatorRegistry.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.reflections.Reflections;
+import org.reflections.scanners.MethodAnnotationsScanner;
+import org.reflections.util.ClasspathHelper;
+import org.reflections.util.ConfigurationBuilder;
+
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Registry that loads and stores the request validators to be applied by
+ * a service.
+ */
+public class ValidatorRegistry {
+
+  private final EnumMap<
+      ValidationCondition, EnumMap<Type, Pair<List<Method>, List<Method>>>>
+      validators = new EnumMap<>(ValidationCondition.class);
+
+  /**
+   * Creates a {@link ValidatorRegistry} instance that discovers validation
+   * methods in the provided package and the packages in the same resource.
+   * A validation method is recognized by the {@link RequestFeatureValidator}
+   * annotation that contains important information about how and when to use
+   * the validator.
+   * @param validatorPackage the main package inside which validatiors should
+   *                         be discovered.
+   */
+  ValidatorRegistry(String validatorPackage) {
+    this(ClasspathHelper.forPackage(validatorPackage));
+  }
+
+  /**
+   * Creates a {@link ValidatorRegistry} instance that discovers validation
+   * methods under the provided URL.
+   * A validation method is recognized by the {@link RequestFeatureValidator}
+   * annotation that contains important information about how and when to use
+   * the validator.
+   * @param searchUrls the path in which the annotated methods are searched.
+   */
+  ValidatorRegistry(Collection<URL> searchUrls) {
+    Reflections reflections = new Reflections(new ConfigurationBuilder()
+        .setUrls(searchUrls)
+        .setScanners(new MethodAnnotationsScanner())
+        .useParallelExecutor()
+    );
+
+    Set<Method> describedValidators =
+        reflections.getMethodsAnnotatedWith(RequestFeatureValidator.class);
+    initMaps(describedValidators);
+  }
+
+  /**
+   * Get the validators that has to be run in the given list of
+   * {@link ValidationCondition}s, for the given requestType and
+   * {@link RequestProcessingPhase}.
+   *
+   * @param conditions conditions that are present for the request
+   * @param requestType the type of the protocol message
+   * @param phase the request processing phase
+   * @return the list of validation methods that has to run.
+   */
+  List<Method> validationsFor(
+      List<ValidationCondition> conditions,
+      Type requestType,
+      RequestProcessingPhase phase) {
+
+    if (conditions.isEmpty() || validators.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    Set<Method> returnValue =
+        new HashSet<>(validationsFor(conditions.get(0), requestType, phase));

Review comment:
       Makes sense, thanks for the explanation.




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] fapifta commented on a change in pull request #3104: HDDS-6213. Pluggable OzoneManager request handling hooks

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #3104:
URL: https://github.com/apache/ozone/pull/3104#discussion_r825120109



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
##########
@@ -0,0 +1,422 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.ValidationListener;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.ozone.upgrade.LayoutVersionManager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.request.validation.ValidationContext.of;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.startValidatorTest;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.finishValidatorTest;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status.OK;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateKey;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.DeleteKeys;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.RenameKey;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Testing the RequestValidations class that is used to run the validation for
+ * any given request that arrives to OzoneManager.
+ */
+public class TestRequestValidations {
+  private static final String PACKAGE =
+      "org.apache.hadoop.ozone.om.request.validation.testvalidatorset1";
+
+  private static final String PACKAGE_WO_VALIDATORS =
+      "org.apache.hadoop.hdds.annotation";
+
+  private final ValidationListenerImpl validationListener =
+      new ValidationListenerImpl();
+
+  @Before
+  public void setup() {
+    startValidatorTest();
+    validationListener.attach();
+  }
+
+  @After
+  public void tearDown() {
+    validationListener.detach();
+    finishValidatorTest();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutLoading() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .withinContext(of(aFinalizedVersionManager()))
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutContext() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test
+  public void testUsingRegistryWithoutPackage() throws ServiceException {
+    new RequestValidations()
+        .withinContext(of(aFinalizedVersionManager()))
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+//    validationListener
+//        .assertCalled("unconditionalPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testNoPreValdiationsWithoutValidationMethods()
+      throws ServiceException {
+    int omVersion = 0;
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadEmptyValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(omVersion));
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPostValdiationsWithoutValidationMethods()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadEmptyValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(currentClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPreValidationsRunningForRequestTypeWithoutValidators()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aRenameKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPostValidationsAreRunningForRequestTypeWithoutValidators()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aRenameKeyRequest(currentClientVersion()), aRenameKeyResponse());
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+/*  @Test

Review comment:
       I have removed the code that was commented out, we already see that this code probably won't last that long, and the adjustments most likely will bring us towards a different direction, so the code won't possibly be used ever, with that it does not worth to preserve it. Worst case we can still pull it off from my fork, I won't delete the branch for a while.

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
##########
@@ -0,0 +1,422 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.ValidationListener;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.ozone.upgrade.LayoutVersionManager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.request.validation.ValidationContext.of;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.startValidatorTest;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.finishValidatorTest;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status.OK;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateKey;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.DeleteKeys;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.RenameKey;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Testing the RequestValidations class that is used to run the validation for
+ * any given request that arrives to OzoneManager.
+ */
+public class TestRequestValidations {
+  private static final String PACKAGE =
+      "org.apache.hadoop.ozone.om.request.validation.testvalidatorset1";
+
+  private static final String PACKAGE_WO_VALIDATORS =
+      "org.apache.hadoop.hdds.annotation";
+
+  private final ValidationListenerImpl validationListener =
+      new ValidationListenerImpl();
+
+  @Before
+  public void setup() {
+    startValidatorTest();
+    validationListener.attach();
+  }
+
+  @After
+  public void tearDown() {
+    validationListener.detach();
+    finishValidatorTest();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutLoading() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .withinContext(of(aFinalizedVersionManager()))
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutContext() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test
+  public void testUsingRegistryWithoutPackage() throws ServiceException {
+    new RequestValidations()
+        .withinContext(of(aFinalizedVersionManager()))
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+//    validationListener
+//        .assertCalled("unconditionalPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testNoPreValdiationsWithoutValidationMethods()

Review comment:
       Fixed, thank you for noting it!

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/validation/TestRequestValidations.java
##########
@@ -0,0 +1,422 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.ozone.om.request.validation;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.ozone.ClientVersions;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting;
+import org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.ValidationListener;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.ozone.upgrade.LayoutVersionManager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.request.validation.ValidationContext.of;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.startValidatorTest;
+import static org.apache.hadoop.ozone.om.request.validation.testvalidatorset1.GeneralValidatorsForTesting.finishValidatorTest;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status.OK;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.CreateKey;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.DeleteKeys;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type.RenameKey;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Testing the RequestValidations class that is used to run the validation for
+ * any given request that arrives to OzoneManager.
+ */
+public class TestRequestValidations {
+  private static final String PACKAGE =
+      "org.apache.hadoop.ozone.om.request.validation.testvalidatorset1";
+
+  private static final String PACKAGE_WO_VALIDATORS =
+      "org.apache.hadoop.hdds.annotation";
+
+  private final ValidationListenerImpl validationListener =
+      new ValidationListenerImpl();
+
+  @Before
+  public void setup() {
+    startValidatorTest();
+    validationListener.attach();
+  }
+
+  @After
+  public void tearDown() {
+    validationListener.detach();
+    finishValidatorTest();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutLoading() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .withinContext(of(aFinalizedVersionManager()))
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUsingRegistryWithoutContext() throws ServiceException {
+    new RequestValidations()
+        .fromPackage(PACKAGE)
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+  }
+
+  @Test
+  public void testUsingRegistryWithoutPackage() throws ServiceException {
+    new RequestValidations()
+        .withinContext(of(aFinalizedVersionManager()))
+        .load()
+        .validateRequest(aCreateKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+//    validationListener
+//        .assertCalled("unconditionalPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testNoPreValdiationsWithoutValidationMethods()
+      throws ServiceException {
+    int omVersion = 0;
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadEmptyValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(omVersion));
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPostValdiationsWithoutValidationMethods()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadEmptyValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(currentClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPreValidationsRunningForRequestTypeWithoutValidators()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aRenameKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+  @Test
+  public void testNoPostValidationsAreRunningForRequestTypeWithoutValidators()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aRenameKeyRequest(currentClientVersion()), aRenameKeyResponse());
+
+    validationListener.assertNumOfEvents(0);
+  }
+
+/*  @Test
+  public void testUnconditionalPreProcessValidationsAreCalled()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(currentClientVersion()));
+
+    validationListener.assertNumOfEvents(1);
+    validationListener
+        .assertCalled("unconditionalPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testUnconditionalPostProcessValidationsAreCalled()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(currentClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(1);
+    validationListener
+        .assertCalled("unconditionalPostProcessCreateKeyValidator");
+  }*/
+
+  @Test
+  public void testPreProcessorExceptionHandling() {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    try {
+      validations.validateRequest(aDeleteKeysRequest(olderClientVersion()));
+      fail("ServiceException was expected but was not thrown.");
+    } catch (ServiceException ignored) { }
+
+    validationListener.assertNumOfEvents(1);
+    validationListener.assertCalled("throwingPreProcessValidator");
+  }
+
+  @Test
+  public void testPostProcessorExceptionHandling() {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    try {
+      validations.validateResponse(
+          aDeleteKeysRequest(olderClientVersion()), aDeleteKeysResponse());
+      fail("ServiceException was expected but was not thrown.");
+    } catch (ServiceException ignored) { }
+
+    validationListener.assertNumOfEvents(1);
+    validationListener.assertCalled("throwingPostProcessValidator");
+  }
+
+/*  @Test
+  public void testNewClientConditionIsRecognizedAndPreValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(newerClientVersion()));
+
+    validationListener.assertNumOfEvents(2);
+    validationListener.assertAllCalled(
+        "unconditionalPreProcessCreateKeyValidator",
+        "newClientPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testNewClientConditionIsRecognizedAndPostValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(newerClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(3);
+    validationListener.assertAllCalled(
+        "unconditionalPostProcessCreateKeyValidator",
+        "newClientPostProcessCreateKeyValidator",
+        "newClientPostProcessCreateKeyValidator2");
+  }*/
+
+  @Test
+  public void testOldClientConditionIsRecognizedAndPreValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(olderClientVersion()));
+
+    validationListener.assertNumOfEvents(1);
+    validationListener.assertAllCalled(
+//        "unconditionalPreProcessCreateKeyValidator",
+        "oldClientPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testOldClientConditionIsRecognizedAndPostValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(aFinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(olderClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(2);
+    validationListener.assertAllCalled(
+//        "unconditionalPostProcessCreateKeyValidator",
+        "oldClientPostProcessCreateKeyValidator",
+        "oldClientPostProcessCreateKeyValidator2");
+  }
+
+  @Test
+  public void testPreFinalizedWithOldClientConditionPreProcValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(anUnfinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateRequest(aCreateKeyRequest(olderClientVersion()));
+
+    validationListener.assertNumOfEvents(2);
+    validationListener.assertAllCalled(
+//        "unconditionalPreProcessCreateKeyValidator",
+        "preFinalizePreProcessCreateKeyValidator",
+        "oldClientPreProcessCreateKeyValidator");
+  }
+
+  @Test
+  public void testPreFinalizedWithOldClientConditionPostProcValidatorsApplied()
+      throws ServiceException {
+    ValidationContext ctx = of(anUnfinalizedVersionManager());
+    RequestValidations validations = loadValidations(ctx);
+
+    validations.validateResponse(
+        aCreateKeyRequest(olderClientVersion()), aCreateKeyResponse());
+
+    validationListener.assertNumOfEvents(3);
+    validationListener.assertAllCalled(
+//        "unconditionalPostProcessCreateKeyValidator",
+        "preFinalizePostProcessCreateKeyValidator",
+        "oldClientPostProcessCreateKeyValidator",
+        "oldClientPostProcessCreateKeyValidator2");
+  }
+
+  private RequestValidations loadValidations(ValidationContext ctx) {
+    return new RequestValidations()
+        .fromPackage(PACKAGE)
+        .withinContext(ctx)
+        .load();
+  }
+
+  private RequestValidations loadEmptyValidations(ValidationContext ctx) {
+    return new RequestValidations()
+        .fromPackage(PACKAGE_WO_VALIDATORS)
+        .withinContext(ctx)
+        .load();
+  }
+
+  private int olderClientVersion() {
+    return ClientVersions.CURRENT_VERSION - 1;
+  }
+
+  private int currentClientVersion() {
+    return ClientVersions.CURRENT_VERSION;
+  }
+
+/*  private int newerClientVersion() {
+    return ClientVersions.CURRENT_VERSION + 1;
+  }*/
+
+  private OMRequest aCreateKeyRequest(int clientVersion) {
+    return aRequest(CreateKey, clientVersion);
+  }
+
+  private OMRequest aDeleteKeysRequest(int clientVersion) {
+    return aRequest(DeleteKeys, clientVersion);
+  }
+
+  private OMRequest aRenameKeyRequest(int clientVersion) {
+    return aRequest(RenameKey, clientVersion);
+  }
+
+  private OMRequest aRequest(Type type, int clientVersion) {
+    return OMRequest.newBuilder()
+        .setVersion(clientVersion)
+        .setCmdType(type)
+        .setClientId("TestClient")
+        .build();
+  }
+
+  private OMResponse aCreateKeyResponse() {
+    return aResponse(CreateKey);
+  }
+
+  private OMResponse aDeleteKeysResponse() {
+    return aResponse(DeleteKeys);
+  }
+
+  private OMResponse aRenameKeyResponse() {
+    return aResponse(RenameKey);
+  }
+
+  private OMResponse aResponse(Type type) {
+    return OMResponse.newBuilder()
+        .setCmdType(type)
+        .setStatus(OK)
+        .build();
+  }
+
+  private LayoutVersionManager aFinalizedVersionManager() {
+    LayoutVersionManager vm = mock(LayoutVersionManager.class);
+    when(vm.needsFinalization()).thenReturn(false);
+    return vm;
+  }
+
+  private LayoutVersionManager anUnfinalizedVersionManager() {
+    LayoutVersionManager vm = mock(LayoutVersionManager.class);
+    when(vm.needsFinalization()).thenReturn(true);
+    return vm;
+  }
+
+  private static class ValidationListenerImpl implements ValidationListener {
+    private List<String> calledMethods = new ArrayList<>();
+
+    @Override
+    public void validationCalled(String calledMethodName) {
+      calledMethods.add(calledMethodName);
+    }
+
+    public void attach() {
+      GeneralValidatorsForTesting.addListener(this);
+    }
+
+    public void detach() {
+      GeneralValidatorsForTesting.removeListener(this);
+      reset();
+    }
+
+    public void reset() {
+      calledMethods = new ArrayList<>();
+    }
+
+    public void assertCalled(String... methodNames) {

Review comment:
       The two method serves a different purpose, this is to check if the methods were called, but does not bother if other methods were also called. This is because I had some troubles to figure out how to restrict Reflections to really just one package instead of the jar file(s) that contains the package.
   I switched to allcalled, and renamed that method to better represent what it does to assertExactListOfValidatorsCalled(String...)




-- 
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: issues-unsubscribe@ozone.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org