You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2019/03/16 12:15:07 UTC

[GitHub] [nifi] MikeThomsen commented on a change in pull request #3165: NIFI-5318 Implement NiFi test harness

MikeThomsen commented on a change in pull request #3165: NIFI-5318 Implement NiFi test harness
URL: https://github.com/apache/nifi/pull/3165#discussion_r266199297
 
 

 ##########
 File path: nifi-testharness/src/main/java/org/apache/nifi/testharness/TestNiFiInstance.java
 ##########
 @@ -0,0 +1,620 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+
+package org.apache.nifi.testharness;
+
+import org.apache.nifi.EmbeddedNiFi;
+import org.apache.nifi.testharness.api.FlowFileEditorCallback;
+import org.apache.nifi.testharness.util.FileUtils;
+import org.apache.nifi.testharness.util.NiFiCoreLibClassLoader;
+import org.apache.nifi.testharness.util.XmlUtils;
+import org.apache.nifi.testharness.util.Zip;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+import javax.xml.xpath.XPathFactory;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+import java.util.zip.ZipEntry;
+
+/**
+ * <p>
+ * An API wrapper of a "test" NiFi instance to which a flow definition is installed for testing.</p>
+ *
+ * <p>
+ * Due to NiFi design restrictions, {@code TestNiFiInstance} has to take <i>full command</i>
+ * of the current working directory: it installs a full NiFi installation to there. To ensure
+ * this is desired, <strong>it will only run if the current directory is called
+ * "nifi_testharness_nifi_home"</strong>. As such the JVM process has to be started inside a directory
+ * called "nifi_testharness_nifi_home" so that the following is true:
+ *
+ * <pre><tt>
+ *     new File(System.getProperty("user.dir")).getName().equals("nifi_testharness_nifi_home")
+ * </tt></pre>
+ * </p>
+ *
+ * <p>
+ * Before {@code TestNiFiInstance} can be used, it has to be configured via its builder
+ * interface:
+ * <ul>
+ *     <li>
+ *      {@link Builder#setNiFiBinaryDistributionZip(File)} specifies the location of the NiFi binary
+ *      distribution ZIP file to be used.
+ *     </li>
+ *      <li>
+ *      {@link Builder#setFlowXmlToInstallForTesting(File)} specifies the location of the NiFi flow
+ *      to install.
+ *     </li>
+ *     <li>
+ *      {@link Builder#modifyFlowXmlBeforeInstalling(FlowFileEditorCallback)} allows on-the-fly
+ *      changes to be performed to the Flow file before it is actually installed.
+ *     </li>
+ * </ul>
+ *
+ * <h5>Sample</h5>
+ * <pre><tt>
+ * TestNiFiInstance testNiFiInstance = TestNiFiInstance.builder()
+ *      .setNiFiBinaryDistributionZip(YourConstants.NIFI_ZIP_FILE)
+ *      .setFlowXmlToInstallForTesting(YourConstants.FLOW_XML_FILE)
+ *      .modifyFlowXmlBeforeInstalling(YourConstants.FLOW_FILE_CHANGES_FOR_TESTS)
+ *      .build();
+ * </tt></pre>
+ *
+ * </p>
+ *
+ * <p>
+ * If the current working directory is called "nifi_testharness_nifi_home", the caller can
+ * {@link #install()} this {@code TestNiFiInstance}, which will
+ * <ol>
+ *  <li>
+ *      (as a first cleanup step) erase all content of the current working directory.
+ *      (NOTE: this potentially destructive operation is the reason why we have the
+ *      "nifi_testharness_nifi_home" directory name guard in place!)
+ *  </li>
+ *  <li>
+ *      Extracts the contents of the NiFi binary distribution ZIP file specified in
+ *      the configuration to a to a temporary directory.
+ *  <li>
+ *      Symlinks all files from the temporary directory to the current working
+ *      directory, causing the directory to hold a fully functional
+ *      NiFi installation.
+ *  </li>
+ *  <li>
+ *      Installs the flow definition files(s) to the NiFi instance specified in
+ *      the configuration.
+ *  </li>
+ * </ol>
+ * </p>
+ *
+ * <p>
+ *
+ * The caller then can proceed to {@link #start()} this {@code TestNiFiInstance},
+ * which will bootstrap the NiFi engine, which in turn will pick up and start processing
+ * the flow definition supplied by the caller in the configuration.
+ * </p>
+ *
+ * <p>
+ * Once the previous step is done, the caller can perform asserts regarding the observed behaviour
+ * of the NiFi flow, just like one would do it with standard Java test cases.
+ * </p>
+ *
+ * <p>
+ * To perform a clean shutdown of the hosted NiFi instance, the caller is required to call
+ * {@link #stopAndCleanup()}, which will shut down NiFi and remove all temporary files, including
+ * the symlinks created in the current working directory.
+ * </p>
+ *
+ *
+ * <h4>NOTES</h4>
+ * <ul>
+ *  <li>
+ *      {@code TestNiFiInstance} is NOT thread safe: if more than one thread uses it,
+ *      external synchronisation is required.
+ *  </li>
+ *   <li>
+ *      Only one {@code TestNiFiInstance} can be started in the same "nifi_testharness_nifi_home"
+ *      directory at the same time.
+ *  </li>
+ *  <li>
+ *      Currently, due to NiFi limitations, one {@code TestNiFiInstance} can be started per JVM process.
+ *      If multiple test cases are required, launch a new JVM process per test case
+ *      (in sequence, see the point above): Maven/Surefire has built-in support for this.
+ *  </li>
+ * </ul>
+ *
+ * <p>
+ * <strong>CAUTION: THIS IS AN EXPERIMENTAL API: EXPECT CHANGES!</strong>
+ * Efforts will be made to retain backwards API compatibility, but
+ * no guarantee is given.
+ * </p>
+ *
+ *
+ * @see TestNiFiInstance#builder()
+ *
+ *
+ */
+public class TestNiFiInstance {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TestNiFiInstance.class);
+
+
+    private EmbeddedNiFi testNiFi;
+
+    private final File nifiHomeDir;
+    private final File bootstrapLibDir;
+
+    private File nifiProperties;
+
+    private final File flowXmlGz;
+
+    private final File placeholderNiFiHomeDir;
+
+    private String nifiVersion;
+
+
+    private enum State {
+        STOPPED,
+        STOP_FAILED,
+        START_FAILED(STOPPED),
+        STARTED(STOPPED, STOP_FAILED),
+        INSTALLATION_FAILED(),
+        FLOW_INSTALLED(STARTED, START_FAILED),
+        INSTALLED(FLOW_INSTALLED, INSTALLATION_FAILED),
+        CREATED(INSTALLED, INSTALLATION_FAILED);
+
+
+        private final Set<State> allowedTransitions;
+
+        State(State... allowedTransitions) {
+            this.allowedTransitions = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(allowedTransitions)));
+        }
+
+        private void checkCanTransition(State newState) {
+            if (!this.allowedTransitions.contains(newState)) {
+                throw new IllegalStateException("Cannot transition from " + this + " to " + newState);
+            }
+        }
+    }
+
+    private State currentState = State.CREATED;
+
+    private final File nifiBinaryZip;
+    private final File flowXml;
+    private final FlowFileEditorCallback editCallback;
+
+    private TestNiFiInstance(File nifiBinaryZip, File flowXml, FlowFileEditorCallback editCallback) {
+        this.nifiBinaryZip = Objects.requireNonNull(nifiBinaryZip, "nifiBinaryZip");
+        this.flowXml = Objects.requireNonNull(flowXml, "flowXml");
+        this.editCallback = editCallback;
+
+        nifiHomeDir = requireCurrentWorkingDirectoryIsCorrect();
+
+        final File configDir = new File(nifiHomeDir, "conf");
+        final File libDir = new File(nifiHomeDir, "lib");
+
+        bootstrapLibDir = new File(libDir, "bootstrap");
+
+        nifiProperties = new File(configDir, "nifi.properties");
+
+        flowXmlGz = new File(configDir, "flow.xml.gz");
+
+        placeholderNiFiHomeDir = requireCurrentWorkingDirectoryIsCorrect();
+    }
+
+    String getNifiVersion() {
+        switch (currentState) {
+            case INSTALLED:
+            case FLOW_INSTALLED:
+            case STARTED:
+            case START_FAILED:
+            case STOP_FAILED:
+            case STOPPED:
+
+                return Objects.requireNonNull(nifiVersion, "nifiVersion is null");
+
+            default:
+                throw new IllegalStateException(
+                        "NiFi version can only be retrieved after a successful installation, not in: "
+                                + currentState);
+        }
+    }
+
+    public void install() throws IOException {
+
+        currentState.checkCanTransition(State.INSTALLED);
+
+        File[] staleInstallations = placeholderNiFiHomeDir.listFiles((dir, name) -> name.startsWith("nifi-"));
+        if (staleInstallations != null) {
+            Arrays.stream(staleInstallations).forEach(TestNiFiInstance::deleteFileOrDirectoryRecursively);
+        }
+
+        Path tempDirectory = null;
+        try {
+            tempDirectory = Files.createTempDirectory("installable-flow");
+
+
+
+            LOGGER.info("Uncompressing NiFi archive {} to {} ...", nifiBinaryZip, placeholderNiFiHomeDir);
+
+            Zip.unzipFile(nifiBinaryZip, placeholderNiFiHomeDir, new Zip.StatusListenerAdapter() {
+                @Override
+                public void onUncompressDone(ZipEntry ze) {
+                    LOGGER.debug("Uncompressed {}", ze.getName());
+                }
+            });
+
+            LOGGER.info("Uncompressing DONE");
+
+            File actualNiFiHomeDir = getActualNiFiHomeDir(placeholderNiFiHomeDir);
+
+            nifiVersion = getNiFiVersion(actualNiFiHomeDir);
+
+            currentState = State.INSTALLED;
+
+            File installableFlowFile = createInstallableFlowFile(tempDirectory);
+
+            validateNiFiVersionAgainstFlowVersion(nifiVersion, installableFlowFile);
+
+            FileUtils.createSymlinks(placeholderNiFiHomeDir, actualNiFiHomeDir);
+
+            installFlowFile(installableFlowFile);
+        } catch (Exception e) {
+
+            currentState = State.INSTALLATION_FAILED;
+
+            throw new RuntimeException("Installation failed: " + e.getMessage(), e);
+
+        } finally {
+            if (tempDirectory != null) {
+                FileUtils.deleteDirectoryRecursive(tempDirectory);
+            }
+        }
+
+        currentState = State.FLOW_INSTALLED;
+    }
+
+    private File createInstallableFlowFile(Path tempDirectory) throws IOException {
+
+        File flowXmlFile = new File(tempDirectory.toFile(), "flow.xml");
+
+        if (editCallback == null) {
+            Files.copy(flowXml.toPath(), flowXmlFile.toPath());
+        } else {
+            if (editCallback instanceof TestNiFiInstanceAware) {
+                ((TestNiFiInstanceAware)editCallback).setTestNiFiInstance(this);
+            }
+
+            XmlUtils.editXml(flowXml, flowXmlFile, editCallback);
+        }
+
+        return flowXmlFile;
+    }
+
+    private void installFlowFile(File fileToIncludeInGz) throws IOException {
+        Zip.gzipFile(fileToIncludeInGz, flowXmlGz);
 
 Review comment:
   @markap14 @bbende Can one of you take a look here at how he's building up the test flow.xml.gz? I have a feeling he's making a bad assumption because the synchronizer code is blowing up in my log file in a place where it never does outside of this use case.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services