You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2020/09/02 16:15:35 UTC

[cassandra] branch trunk updated: add nodetool getfullquerylog command

This is an automated email from the ASF dual-hosted git repository.

brandonwilliams pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 419099c  add nodetool getfullquerylog command
419099c is described below

commit 419099c92b1cecb7d22fba99ae8ac2cce4f39638
Author: Stefan Miklosovic <st...@instaclustr.com>
AuthorDate: Fri Aug 28 12:44:48 2020 +0200

    add nodetool getfullquerylog command
    
    Patch by Stefan Miklosovic; reviewed by Berenguer Blasi, brandonwilliams, and dcapwell for
    CASSANDRA-15988
---
 CHANGES.txt                                        |   1 +
 doc/source/new/fqllogging.rst                      |  19 +++
 .../org/apache/cassandra/fql/FullQueryLogger.java  |  27 ++++
 .../fql/FullQueryLoggerOptionsCompositeData.java   | 115 +++++++++++++++++
 .../apache/cassandra/service/StorageService.java   |  13 ++
 .../cassandra/service/StorageServiceMBean.java     |   8 +-
 src/java/org/apache/cassandra/tools/NodeProbe.java |   7 +
 src/java/org/apache/cassandra/tools/NodeTool.java  |   1 +
 .../cassandra/tools/nodetool/GetFullQueryLog.java  |  48 +++++++
 .../org/apache/cassandra/utils/binlog/BinLog.java  |  41 ++++--
 .../cassandra/tools/GetFullQueryLogTest.java       | 141 +++++++++++++++++++++
 11 files changed, 407 insertions(+), 14 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index a7ed819..14df925 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-beta3
+ * Add nodetool getfullquerylog (CASSANDRA-15988)
 
 4.0-beta2
  * Add addition incremental repair visibility to nodetool repair_admin (CASSANDRA-14939)
diff --git a/doc/source/new/fqllogging.rst b/doc/source/new/fqllogging.rst
index 5a78931..daead9c 100644
--- a/doc/source/new/fqllogging.rst
+++ b/doc/source/new/fqllogging.rst
@@ -171,6 +171,25 @@ For example: ``max_archive_retries: 10``
 
 FQL can also be configured using ``nodetool`` when enabling the feature, and will override any values set in the `cassandra.yaml` file, as discussed in the next section.
 
+Querying the state of FQL
+---------------------
+
+In order to know what state FQL is in, you may use nodetool command ``getfullquerylog``. It will print out whether FQL is enabled
+and with what configuration options; if you reset or stop FQL, the configuration displayed will be taken from
+configuration in ``cassandra.yaml``.
+
+::
+
+ $ nodetool getfullquerylog
+ enabled             true
+ log_dir             /path/to/fql/log/dir
+ archive_command     /usr/local/bin/archiveit.sh %path
+ roll_cycle          HOURLY
+ block               true
+ max_log_size        17179869184
+ max_queue_weight    268435456
+ max_archive_retries 10
+
 Enabling FQL
 ------------
 
diff --git a/src/java/org/apache/cassandra/fql/FullQueryLogger.java b/src/java/org/apache/cassandra/fql/FullQueryLogger.java
index 6c38166..4b40635 100644
--- a/src/java/org/apache/cassandra/fql/FullQueryLogger.java
+++ b/src/java/org/apache/cassandra/fql/FullQueryLogger.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.fql;
 import java.io.File;
 import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
@@ -37,6 +38,7 @@ import io.netty.buffer.ByteBuf;
 import net.openhft.chronicle.bytes.BytesStore;
 import net.openhft.chronicle.wire.ValueOut;
 import net.openhft.chronicle.wire.WireOut;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryEvents;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -46,6 +48,7 @@ import org.apache.cassandra.transport.CBUtil;
 import org.apache.cassandra.transport.Message;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.binlog.BinLog;
+import org.apache.cassandra.utils.binlog.BinLogOptions;
 import org.apache.cassandra.utils.concurrent.WeightedQueue;
 import org.github.jamm.MemoryLayoutSpecification;
 
@@ -117,6 +120,30 @@ public class FullQueryLogger implements QueryEvents.Listener
         }
     }
 
+    public FullQueryLoggerOptions getFullQueryLoggerOptions()
+    {
+        if (isEnabled())
+        {
+            final FullQueryLoggerOptions options = new FullQueryLoggerOptions();
+            final BinLogOptions binLogOptions = binLog.getBinLogOptions();
+
+            options.archive_command = binLogOptions.archive_command;
+            options.roll_cycle = binLogOptions.roll_cycle;
+            options.block = binLogOptions.block;
+            options.max_archive_retries = binLogOptions.max_archive_retries;
+            options.max_queue_weight = binLogOptions.max_queue_weight;
+            options.max_log_size = binLogOptions.max_log_size;
+            options.log_dir = binLog.path.toString();
+
+            return options;
+        }
+        else
+        {
+            // otherwise get what database is configured with from cassandra.yaml
+            return DatabaseDescriptor.getFullQueryLogOptions();
+        }
+    }
+
     public synchronized void stop()
     {
         try
diff --git a/src/java/org/apache/cassandra/fql/FullQueryLoggerOptionsCompositeData.java b/src/java/org/apache/cassandra/fql/FullQueryLoggerOptionsCompositeData.java
new file mode 100644
index 0000000..184f184
--- /dev/null
+++ b/src/java/org/apache/cassandra/fql/FullQueryLoggerOptionsCompositeData.java
@@ -0,0 +1,115 @@
+/*
+ * 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.cassandra.fql;
+
+import java.util.HashMap;
+import java.util.Map;
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.CompositeDataSupport;
+import javax.management.openmbean.CompositeType;
+import javax.management.openmbean.OpenDataException;
+import javax.management.openmbean.OpenType;
+import javax.management.openmbean.SimpleType;
+
+public class FullQueryLoggerOptionsCompositeData
+{
+    private static final String[] ITEM_NAMES = new String[]{ "log_dir",
+                                                             "archive_command",
+                                                             "roll_cycle",
+                                                             "block",
+                                                             "max_queue_weight",
+                                                             "max_log_size",
+                                                             "max_archive_retries" };
+
+    private static final String[] ITEM_DESC = new String[]{ "directory where FQL data are stored",
+                                                            "archive command for FQL data",
+                                                            "how often to roll BinLog segments so they can potentially be reclaimed",
+                                                            "indicates if the BinLog should block if the it falls behind or should drop bin log records",
+                                                            "maximum weight of in memory queue for records waiting to be written to the binlog file before blocking or dropping the log records",
+                                                            "maximum size of the rolled files to retain on disk before deleting the oldest file",
+                                                            "number of times to retry an archive command" };
+
+    private static final OpenType<?>[] ITEM_TYPES;
+
+    public static final CompositeType COMPOSITE_TYPE;
+
+    static
+    {
+        try
+        {
+            ITEM_TYPES = new OpenType[]{ SimpleType.STRING,
+                                         SimpleType.STRING,
+                                         SimpleType.STRING,
+                                         SimpleType.BOOLEAN,
+                                         SimpleType.INTEGER,
+                                         SimpleType.LONG,
+                                         SimpleType.INTEGER };
+
+            COMPOSITE_TYPE = new CompositeType(FullQueryLoggerOptions.class.getName(),
+                                               "FullQueryLoggerOptions",
+                                               ITEM_NAMES,
+                                               ITEM_DESC,
+                                               ITEM_TYPES);
+        }
+        catch (final OpenDataException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static CompositeData toCompositeData(final FullQueryLoggerOptions options)
+    {
+        Map<String, Object> valueMap = new HashMap<>();
+        valueMap.put(ITEM_NAMES[0], options.log_dir);
+        valueMap.put(ITEM_NAMES[1], options.archive_command);
+        valueMap.put(ITEM_NAMES[2], options.roll_cycle);
+        valueMap.put(ITEM_NAMES[3], options.block);
+        valueMap.put(ITEM_NAMES[4], options.max_queue_weight);
+        valueMap.put(ITEM_NAMES[5], options.max_log_size);
+        valueMap.put(ITEM_NAMES[6], options.max_archive_retries);
+
+        try
+        {
+            return new CompositeDataSupport(COMPOSITE_TYPE, valueMap);
+        }
+        catch (final OpenDataException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static FullQueryLoggerOptions fromCompositeData(final CompositeData data)
+    {
+        assert data.getCompositeType().equals(COMPOSITE_TYPE);
+
+        final Object[] values = data.getAll(ITEM_NAMES);
+
+        final FullQueryLoggerOptions options = new FullQueryLoggerOptions();
+
+        options.log_dir = (String) values[0];
+        options.archive_command = (String) values[1];
+        options.roll_cycle = (String) values[2];
+        options.block = (Boolean) values[3];
+        options.max_queue_weight = (Integer) values[4];
+        options.max_log_size = (Long) values[5];
+        options.max_archive_retries = (Integer) values[6];
+
+        return options;
+    }
+}
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index f7c0441..c06b819 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -51,6 +51,7 @@ import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
 import org.apache.cassandra.fql.FullQueryLogger;
 import org.apache.cassandra.fql.FullQueryLoggerOptions;
+import org.apache.cassandra.fql.FullQueryLoggerOptionsCompositeData;
 import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 import org.apache.commons.lang3.StringUtils;
 
@@ -5625,4 +5626,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         FullQueryLogger.instance.stop();
     }
+
+    @Override
+    public boolean isFullQueryLogEnabled()
+    {
+        return FullQueryLogger.instance.isEnabled();
+    }
+
+    @Override
+    public CompositeData getFullQueryLoggerOptions()
+    {
+        return FullQueryLoggerOptionsCompositeData.toCompositeData(FullQueryLogger.instance.getFullQueryLoggerOptions());
+    }
 }
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 3c6bbf9..58402e1 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -34,7 +34,6 @@ import javax.management.openmbean.TabularData;
 
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.utils.Pair;
 
 public interface StorageServiceMBean extends NotificationEmitter
 {
@@ -786,6 +785,13 @@ public interface StorageServiceMBean extends NotificationEmitter
      */
     public void stopFullQueryLogger();
 
+    public boolean isFullQueryLogEnabled();
+
+    /**
+     * Returns the current state of FQL.
+     */
+    CompositeData getFullQueryLoggerOptions();
+
     /** Sets the initial allocation size of backing arrays for new RangeTombstoneList objects */
     public void setInitialRangeTombstoneListAllocationSize(int size);
 
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 617da12..685c153 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -62,6 +62,8 @@ import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.CompactionManagerMBean;
+import org.apache.cassandra.fql.FullQueryLoggerOptions;
+import org.apache.cassandra.fql.FullQueryLoggerOptionsCompositeData;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.gms.FailureDetectorMBean;
 import org.apache.cassandra.gms.Gossiper;
@@ -1816,6 +1818,11 @@ public class NodeProbe implements AutoCloseable
     {
         ssProxy.resetFullQueryLogger();
     }
+
+    public FullQueryLoggerOptions getFullQueryLoggerOptions()
+    {
+        return FullQueryLoggerOptionsCompositeData.fromCompositeData(ssProxy.getFullQueryLoggerOptions());
+    }
 }
 
 class ColumnFamilyStoreMBeanIterator implements Iterator<Map.Entry<String, ColumnFamilyStoreMBean>>
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index 69cd04c..f558267 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -127,6 +127,7 @@ public class NodeTool
                 GetCompactionThreshold.class,
                 GetCompactionThroughput.class,
                 GetConcurrency.class,
+                GetFullQueryLog.class,
                 GetTimeout.class,
                 GetStreamThroughput.class,
                 GetTraceProbability.class,
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetFullQueryLog.java b/src/java/org/apache/cassandra/tools/nodetool/GetFullQueryLog.java
new file mode 100644
index 0000000..47d8b8a
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetFullQueryLog.java
@@ -0,0 +1,48 @@
+/*
+ * 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.cassandra.tools.nodetool;
+
+import io.airlift.airline.Command;
+import org.apache.cassandra.fql.FullQueryLoggerOptions;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+import org.apache.cassandra.tools.nodetool.formatter.TableBuilder;
+
+@Command(name = "getfullquerylog", description = "print configuration of fql if enabled, otherwise the configuration reflected in cassandra.yaml")
+public class GetFullQueryLog extends NodeToolCmd
+{
+    protected void execute(NodeProbe probe)
+    {
+        final TableBuilder tableBuilder = new TableBuilder();
+
+        tableBuilder.add("enabled", Boolean.toString(probe.getStorageService().isFullQueryLogEnabled()));
+
+        final FullQueryLoggerOptions options = probe.getFullQueryLoggerOptions();
+
+        tableBuilder.add("log_dir", options.log_dir);
+        tableBuilder.add("archive_command", options.archive_command);
+        tableBuilder.add("roll_cycle", options.roll_cycle);
+        tableBuilder.add("block", Boolean.toString(options.block));
+        tableBuilder.add("max_log_size", Long.toString(options.max_log_size));
+        tableBuilder.add("max_queue_weight", Integer.toString(options.max_queue_weight));
+        tableBuilder.add("max_archive_retries", Long.toString(options.max_archive_retries));
+
+        tableBuilder.printTo(System.out);
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/binlog/BinLog.java b/src/java/org/apache/cassandra/utils/binlog/BinLog.java
index f0a24cf..31108c7 100644
--- a/src/java/org/apache/cassandra/utils/binlog/BinLog.java
+++ b/src/java/org/apache/cassandra/utils/binlog/BinLog.java
@@ -38,11 +38,11 @@ import org.slf4j.LoggerFactory;
 import net.openhft.chronicle.queue.ChronicleQueue;
 import net.openhft.chronicle.queue.ChronicleQueueBuilder;
 import net.openhft.chronicle.queue.ExcerptAppender;
-import net.openhft.chronicle.queue.RollCycle;
 import net.openhft.chronicle.queue.RollCycles;
 import net.openhft.chronicle.wire.WireOut;
 import net.openhft.chronicle.wire.WriteMarshallable;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.fql.FullQueryLoggerOptions;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -83,6 +83,8 @@ public class BinLog implements Runnable
 
     private final AtomicLong droppedSamplesSinceLastLog = new AtomicLong();
 
+    private BinLogOptions options;
+
     /*
     This set contains all the paths we are currently logging to, it is used to make sure
     we don't start writing audit and full query logs to the same path.
@@ -116,26 +118,28 @@ public class BinLog implements Runnable
 
     private volatile boolean shouldContinue = true;
 
-    /**
-     * @param path           Path to store the BinLog. Can't be shared with anything else.
-     * @param rollCycle      How often to roll the log file so it can potentially be deleted
-     * @param maxQueueWeight Maximum weight of in memory queue for records waiting to be written to the file before blocking or dropping
-     */
-    private BinLog(Path path, RollCycle rollCycle, int maxQueueWeight, BinLogArchiver archiver, boolean blocking)
+    private BinLog(Path path, BinLogOptions options, BinLogArchiver archiver)
     {
         Preconditions.checkNotNull(path, "path was null");
-        Preconditions.checkNotNull(rollCycle, "rollCycle was null");
-        Preconditions.checkArgument(maxQueueWeight > 0, "maxQueueWeight must be > 0");
+        Preconditions.checkNotNull(options.roll_cycle, "roll_cycle was null");
+        Preconditions.checkArgument(options.max_queue_weight > 0, "max_queue_weight must be > 0");
         ChronicleQueueBuilder builder = ChronicleQueueBuilder.single(path.toFile());
-        builder.rollCycle(rollCycle);
+        builder.rollCycle(RollCycles.valueOf(options.roll_cycle));
 
-        sampleQueue = new WeightedQueue<>(maxQueueWeight);
+        sampleQueue = new WeightedQueue<>(options.max_queue_weight);
         this.archiver = archiver;
         builder.storeFileListener(this.archiver);
         queue = builder.build();
         appender = queue.acquireAppender();
-        this.blocking = blocking;
+        this.blocking = options.block;
         this.path = path;
+
+        this.options = options;
+    }
+
+    public BinLogOptions getBinLogOptions()
+    {
+        return options;
     }
 
     /**
@@ -360,6 +364,7 @@ public class BinLog implements Runnable
             Preconditions.checkNotNull(path, "path was null");
             File pathAsFile = path.toFile();
             //Exists and is a directory or can be created
+            Preconditions.checkArgument(!pathAsFile.toString().isEmpty(), "you might have forgotten to specify a directory to save logs");
             Preconditions.checkArgument((pathAsFile.exists() && pathAsFile.isDirectory()) || (!pathAsFile.exists() && pathAsFile.mkdirs()), "path exists and is not a directory or couldn't be created");
             Preconditions.checkArgument(pathAsFile.canRead() && pathAsFile.canWrite() && pathAsFile.canExecute(), "path is not readable, writable, and executable");
             this.path = path;
@@ -433,7 +438,17 @@ public class BinLog implements Runnable
                         }
                     }
                 }
-                BinLog binlog = new BinLog(path, RollCycles.valueOf(rollCycle), maxQueueWeight, archiver, blocking);
+
+                final BinLogOptions options = new BinLogOptions();
+
+                options.max_log_size = maxLogSize;
+                options.max_queue_weight = maxQueueWeight;
+                options.block = blocking;
+                options.roll_cycle = rollCycle;
+                options.archive_command = archiveCommand;
+                options.max_archive_retries = maxArchiveRetries;
+
+                BinLog binlog = new BinLog(path, options, archiver);
                 binlog.start();
                 return binlog;
             }
diff --git a/test/unit/org/apache/cassandra/tools/GetFullQueryLogTest.java b/test/unit/org/apache/cassandra/tools/GetFullQueryLogTest.java
new file mode 100644
index 0000000..82d5482
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/GetFullQueryLogTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.cassandra.tools;
+
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.fql.FullQueryLoggerOptions;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class GetFullQueryLogTest extends CQLTester
+{
+    private static NodeProbe probe;
+    private ToolRunner.Runners runner = new ToolRunner.Runners();
+
+    @ClassRule
+    public static TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        startJMXServer();
+        probe = new NodeProbe(jmxHost, jmxPort);
+    }
+
+    @AfterClass
+    public static void teardown() throws IOException
+    {
+        probe.close();
+    }
+
+    @After
+    public void afterTest()
+    {
+        disableFullQueryLog();
+    }
+
+    @Test
+    public void getFullQueryLogTest()
+    {
+        testDefaultOutput(getFullQueryLog());
+    }
+
+    @Test
+    public void enableFullQueryLogTest()
+    {
+        enableFullQueryLog();
+        testChangedOutput(getFullQueryLog());
+    }
+
+    @Test
+    public void resetFullQueryLogTest()
+    {
+        enableFullQueryLog();
+        testChangedOutput(getFullQueryLog());
+
+        // reset and get and test that it reset configuration into defaults
+        resetFullQueryLog();
+
+        testDefaultOutput(getFullQueryLog());
+    }
+
+    private String getFullQueryLog()
+    {
+        return runner.invokeNodetool("getfullquerylog").waitAndAssertOnCleanExit().getStdout();
+    }
+
+    private void resetFullQueryLog()
+    {
+        runner.invokeNodetool("resetfullquerylog").waitAndAssertOnCleanExit();
+    }
+
+    private void disableFullQueryLog()
+    {
+        runner.invokeNodetool("disablefullquerylog").waitAndAssertOnCleanExit();
+    }
+
+    private void enableFullQueryLog()
+    {
+        runner.invokeNodetool("enablefullquerylog",
+                              "--path", temporaryFolder.getRoot().toString(),
+                              "--blocking", "false",
+                              "--max-archive-retries", "5",
+                              "--archive-command", "/path/to/script.sh %path",
+                              "--max-log-size", "100000",
+                              "--max-queue-weight", "10000",
+                              "--roll-cycle", "DAILY").waitAndAssertOnCleanExit();
+    }
+
+    private void testChangedOutput(final String getFullQueryLogOutput)
+    {
+        final String output = getFullQueryLogOutput.replaceAll("( )+", " ").trim();
+        assertTrue(output.contains("enabled true"));
+        assertTrue(output.contains("log_dir " + temporaryFolder.getRoot().toString()));
+        assertTrue(output.contains("archive_command /path/to/script.sh %path"));
+        assertTrue(output.contains("roll_cycle DAILY"));
+        assertTrue(output.contains("max_log_size 100000"));
+        assertTrue(output.contains("max_queue_weight 10000"));
+        assertTrue(output.contains("max_archive_retries 5"));
+        assertTrue(output.contains("block false"));
+    }
+
+    private void testDefaultOutput(final String getFullQueryLogOutput)
+    {
+        final FullQueryLoggerOptions options = new FullQueryLoggerOptions();
+        final String output = getFullQueryLogOutput.replaceAll("( )+", " ").trim();
+        assertTrue(output.contains("enabled false"));
+        assertFalse(output.contains("log_dir " + temporaryFolder.getRoot().toString()));
+        assertFalse(output.contains("archive_command /path/to/script.sh %path"));
+        assertTrue(output.contains("roll_cycle " + options.roll_cycle));
+        assertTrue(output.contains("max_log_size " + options.max_log_size));
+        assertTrue(output.contains("max_queue_weight " + options.max_queue_weight));
+        assertTrue(output.contains("max_archive_retries " + options.max_archive_retries));
+        assertTrue(output.contains("block " + options.block));
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org