You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by jb...@apache.org on 2016/06/29 20:54:58 UTC

[3/5] activemq-artemis git commit: ARTEMIS-599 add --f option to ignore locking of the server

ARTEMIS-599 add --f option to ignore locking of the server


Project: http://git-wip-us.apache.org/repos/asf/activemq-artemis/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq-artemis/commit/b2de7ded
Tree: http://git-wip-us.apache.org/repos/asf/activemq-artemis/tree/b2de7ded
Diff: http://git-wip-us.apache.org/repos/asf/activemq-artemis/diff/b2de7ded

Branch: refs/heads/master
Commit: b2de7ded1d51cc6953369f1ef3b32033e597a1ea
Parents: a2f936c
Author: Clebert Suconic <cl...@apache.org>
Authored: Wed Jun 29 14:38:59 2016 -0400
Committer: Clebert Suconic <cl...@apache.org>
Committed: Wed Jun 29 16:14:26 2016 -0400

----------------------------------------------------------------------
 .../artemis/cli/commands/Configurable.java      | 43 ---------------
 .../activemq/artemis/cli/commands/Run.java      |  5 +-
 .../cli/commands/tools/LockAbstract.java        | 58 ++++++++++++++++++++
 .../cli/commands/tools/OptionalLocking.java     | 37 +++++++++++++
 .../artemis/cli/commands/tools/PrintData.java   |  2 +-
 .../cli/commands/tools/XmlDataExporter.java     |  2 +-
 .../apache/activemq/cli/test/ArtemisTest.java   | 13 ++++-
 .../artemis/maven/ArtemisAbstractPlugin.java    |  4 +-
 8 files changed, 112 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b2de7ded/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Configurable.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Configurable.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Configurable.java
index 8795cf3..1fd183c 100644
--- a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Configurable.java
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Configurable.java
@@ -19,8 +19,6 @@ package org.apache.activemq.artemis.cli.commands;
 
 import javax.inject.Inject;
 import java.io.File;
-import java.io.RandomAccessFile;
-import java.nio.channels.FileLock;
 
 import io.airlift.airline.Arguments;
 import io.airlift.airline.Help;
@@ -28,7 +26,6 @@ import io.airlift.airline.Option;
 import io.airlift.airline.model.CommandGroupMetadata;
 import io.airlift.airline.model.CommandMetadata;
 import io.airlift.airline.model.GlobalMetadata;
-import org.apache.activemq.artemis.cli.CLIException;
 import org.apache.activemq.artemis.core.config.FileDeploymentManager;
 import org.apache.activemq.artemis.core.config.impl.FileConfiguration;
 import org.apache.activemq.artemis.dto.BrokerDTO;
@@ -75,46 +72,6 @@ public abstract class Configurable extends ActionAbstract {
       }
    }
 
-   // There should be one lock per VM
-   // These will be locked as long as the VM is running
-   private static RandomAccessFile serverLockFile = null;
-   private static FileLock serverLockLock = null;
-
-   protected static void lockCLI(File lockPlace) throws Exception {
-      if (lockPlace != null) {
-         lockPlace.mkdirs();
-         File fileLock = new File(lockPlace, "cli.lock");
-         RandomAccessFile file = new RandomAccessFile(fileLock, "rw");
-         serverLockLock = file.getChannel().tryLock();
-         if (serverLockLock == null) {
-            throw new CLIException("Error: There is another process using the server at " + lockPlace + ". Cannot start the process!");
-         }
-      }
-   }
-
-   protected File getLockPlace() throws Exception {
-      String brokerInstance = getBrokerInstance();
-      if (brokerInstance != null) {
-         return new File(new File(brokerInstance),"lock");
-      }
-      else {
-         return null;
-      }
-   }
-
-   public static void unlock() {
-      try {
-         if (serverLockFile != null) {
-            serverLockFile.close();
-         }
-
-         if (serverLockLock != null) {
-            serverLockLock.close();
-         }
-      }
-      catch (Exception ignored) {
-      }
-   }
 
    protected FileConfiguration getFileConfiguration() throws Exception {
       if (fileConfiguration == null) {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b2de7ded/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java
index c2db6f8..d3c711d 100644
--- a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java
@@ -24,6 +24,7 @@ import java.util.TimerTask;
 import io.airlift.airline.Command;
 import io.airlift.airline.Option;
 import org.apache.activemq.artemis.cli.Artemis;
+import org.apache.activemq.artemis.cli.commands.tools.LockAbstract;
 import org.apache.activemq.artemis.components.ExternalComponent;
 import org.apache.activemq.artemis.core.config.impl.FileConfiguration;
 import org.apache.activemq.artemis.core.server.ActiveMQComponent;
@@ -37,7 +38,7 @@ import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager;
 import org.apache.activemq.artemis.utils.ReusableLatch;
 
 @Command(name = "run", description = "runs the broker instance")
-public class Run extends Configurable {
+public class Run extends LockAbstract {
 
    @Option(name = "--allow-kill", description = "This will allow the server to kill itself. Useful for tests (failover tests for instance)")
    boolean allowKill;
@@ -66,8 +67,6 @@ public class Run extends Configurable {
 
       FileConfiguration fileConfiguration = getFileConfiguration();
 
-      lockCLI(getLockPlace());
-
       Artemis.printBanner();
 
       createDirectories(getFileConfiguration());

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b2de7ded/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/LockAbstract.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/LockAbstract.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/LockAbstract.java
index 4a2a99f..2089d3e 100644
--- a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/LockAbstract.java
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/LockAbstract.java
@@ -17,10 +17,47 @@
 
 package org.apache.activemq.artemis.cli.commands.tools;
 
+import java.io.File;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
+
+import org.apache.activemq.artemis.cli.CLIException;
 import org.apache.activemq.artemis.cli.commands.Action;
 import org.apache.activemq.artemis.cli.commands.ActionContext;
 
 public abstract class LockAbstract  extends DataAbstract implements Action {
+   // There should be one lock per VM
+   // These will be locked as long as the VM is running
+   private static RandomAccessFile serverLockFile = null;
+   private static FileLock serverLockLock = null;
+
+   protected File getLockPlace() throws Exception {
+      String brokerInstance = getBrokerInstance();
+      if (brokerInstance != null) {
+         return new File(new File(brokerInstance),"lock");
+      }
+      else {
+         return null;
+      }
+   }
+
+   public static void unlock() {
+      try {
+         if (serverLockFile != null) {
+            serverLockFile.close();
+            serverLockFile = null;
+         }
+
+         if (serverLockLock != null) {
+            serverLockLock.close();
+            serverLockLock = null;
+         }
+      }
+      catch (Exception ignored) {
+      }
+   }
+
    @Override
    public Object execute(ActionContext context) throws Exception {
       super.execute(context);
@@ -37,4 +74,25 @@ public abstract class LockAbstract  extends DataAbstract implements Action {
    }
 
 
+   protected void lockCLI(File lockPlace) throws Exception {
+      if (lockPlace != null) {
+         lockPlace.mkdirs();
+         if (serverLockFile == null) {
+            File fileLock = new File(lockPlace, "cli.lock");
+            serverLockFile = new RandomAccessFile(fileLock, "rw");
+         }
+         try {
+            FileLock lock = serverLockFile.getChannel().tryLock();
+            if (lock == null) {
+               throw new CLIException("Error: There is another process using the server at " + lockPlace + ". Cannot start the process!");
+            }
+            serverLockLock = lock;
+         }
+         catch (OverlappingFileLockException e) {
+            throw new CLIException("Error: There is another process using the server at " + lockPlace + ". Cannot start the process!");
+         }
+      }
+   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b2de7ded/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/OptionalLocking.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/OptionalLocking.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/OptionalLocking.java
new file mode 100644
index 0000000..5a53656
--- /dev/null
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/OptionalLocking.java
@@ -0,0 +1,37 @@
+/**
+ * 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.activemq.artemis.cli.commands.tools;
+
+import java.io.File;
+
+import io.airlift.airline.Option;
+
+/** This is for commands where --f on ignoring lock could be valid. */
+public class OptionalLocking extends LockAbstract {
+
+   @Option(name = "--f", description = "This will allow certain tools like print-data to be performed ignoring any running servers. WARNING: Changing data concurrently with a running broker may damage your data. Be careful with this option.")
+   boolean ignoreLock;
+
+   @Override
+   protected void lockCLI(File lockPlace) throws Exception {
+      if (!ignoreLock) {
+         super.lockCLI(lockPlace);
+      }
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b2de7ded/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/PrintData.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/PrintData.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/PrintData.java
index 7c5fddf..9b273b2 100644
--- a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/PrintData.java
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/PrintData.java
@@ -58,7 +58,7 @@ import org.apache.activemq.artemis.utils.ActiveMQThreadFactory;
 import org.apache.activemq.artemis.utils.ExecutorFactory;
 
 @Command(name = "print", description = "Print data records information (WARNING: don't use while a production server is running)")
-public class PrintData extends LockAbstract {
+public class PrintData extends OptionalLocking {
 
    @Override
    public Object execute(ActionContext context) throws Exception {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b2de7ded/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataExporter.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataExporter.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataExporter.java
index 3aec34f..05f1fb5 100644
--- a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataExporter.java
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/tools/XmlDataExporter.java
@@ -92,7 +92,7 @@ import org.apache.activemq.artemis.utils.Base64;
 import org.apache.activemq.artemis.utils.ExecutorFactory;
 
 @Command(name = "exp", description = "Export all message-data using an XML that could be interpreted by any system.")
-public final class XmlDataExporter extends LockAbstract {
+public final class XmlDataExporter extends OptionalLocking {
 
    private static final Long LARGE_MESSAGE_CHUNK_SIZE = 1000L;
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b2de7ded/artemis-cli/src/test/java/org/apache/activemq/cli/test/ArtemisTest.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/test/java/org/apache/activemq/cli/test/ArtemisTest.java b/artemis-cli/src/test/java/org/apache/activemq/cli/test/ArtemisTest.java
index 2b87486..b3a9e29 100644
--- a/artemis-cli/src/test/java/org/apache/activemq/cli/test/ArtemisTest.java
+++ b/artemis-cli/src/test/java/org/apache/activemq/cli/test/ArtemisTest.java
@@ -28,9 +28,10 @@ import org.apache.activemq.artemis.api.core.client.ClientSession;
 import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
 import org.apache.activemq.artemis.api.core.client.ServerLocator;
 import org.apache.activemq.artemis.cli.Artemis;
-import org.apache.activemq.artemis.cli.commands.Configurable;
+import org.apache.activemq.artemis.cli.CLIException;
 import org.apache.activemq.artemis.cli.commands.Create;
 import org.apache.activemq.artemis.cli.commands.Run;
+import org.apache.activemq.artemis.cli.commands.tools.LockAbstract;
 import org.apache.activemq.artemis.cli.commands.util.SyncCalculation;
 import org.apache.activemq.artemis.core.client.impl.ServerLocatorImpl;
 import org.apache.activemq.artemis.jlibaio.LibaioContext;
@@ -76,7 +77,7 @@ public class ArtemisTest {
          System.setProperty("java.security.auth.login.config", original);
       }
 
-      Configurable.unlock();
+      LockAbstract.unlock();
    }
 
    @Test
@@ -140,6 +141,14 @@ public class ArtemisTest {
             }
          }
 
+         try {
+            Artemis.internalExecute("data", "print");
+            Assert.fail("Exception expected");
+         }
+         catch (CLIException expected) {
+         }
+         Artemis.internalExecute("data", "print", "--f");
+
          Assert.assertEquals(Integer.valueOf(100), Artemis.internalExecute("producer", "--message-count", "100", "--verbose", "--user", "admin", "--password", "admin"));
          Assert.assertEquals(Integer.valueOf(100), Artemis.internalExecute("consumer", "--verbose", "--break-on-null", "--receive-timeout", "100", "--user", "admin", "--password", "admin"));
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b2de7ded/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ArtemisAbstractPlugin.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ArtemisAbstractPlugin.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ArtemisAbstractPlugin.java
index d4c8abd..8a11857 100644
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ArtemisAbstractPlugin.java
+++ b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ArtemisAbstractPlugin.java
@@ -26,7 +26,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.activemq.artemis.cli.commands.Configurable;
+import org.apache.activemq.artemis.cli.commands.tools.OptionalLocking;
 import org.apache.maven.artifact.repository.ArtifactRepository;
 import org.apache.maven.plugin.AbstractMojo;
 import org.apache.maven.plugin.MojoExecutionException;
@@ -76,7 +76,7 @@ public abstract class ArtemisAbstractPlugin extends AbstractMojo {
          // We could execute the maven plugins over and over on examples
          // For that reason we just unlock the server here
          // Notice this has no implementations if you are using spawn
-         Configurable.unlock();
+         OptionalLocking.unlock();
       }
    }