You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2022/04/12 14:46:40 UTC

[GitHub] [hbase] Apache9 opened a new pull request, #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Apache9 opened a new pull request, #4340:
URL: https://github.com/apache/hbase/pull/4340

   …t output


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache9 commented on a diff in pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4340:
URL: https://github.com/apache/hbase/pull/4340#discussion_r848601196


##########
hbase-logging/src/test/java/org/apache/hadoop/hbase/logging/HBaseTestAppender.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.hbase.logging;
+
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.ManagerFactory;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.apache.logging.log4j.core.appender.rolling.FileSize;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
+import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required;
+
+/**
+ * Log4j2 appender to be used when running UTs.
+ * <p/>
+ * The main point here is to limit the total output size to prevent eating all the space of our ci
+ * system when something wrong in our code.
+ * <p/>
+ * See HBASE-26947 for more details.
+ */
+@Plugin(name = HBaseTestAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+public final class HBaseTestAppender extends AbstractOutputStreamAppender<OutputStreamManager> {
+
+  public static final String PLUGIN_NAME = "HBaseTest";
+  private static final HBaseTestManagerFactory FACTORY = new HBaseTestManagerFactory();
+
+  public static class Builder<B extends Builder<B>> extends AbstractOutputStreamAppender.Builder<B>
+    implements org.apache.logging.log4j.core.util.Builder<HBaseTestAppender> {
+
+    @PluginBuilderAttribute
+    @Required
+    private Target target;
+
+    @PluginBuilderAttribute
+    @Required
+    private String maxSize;
+
+    public B setTarget(Target target) {
+      this.target = target;
+      return asBuilder();
+    }
+
+    public B setMaxSize(String maxSize) {
+      this.maxSize = maxSize;
+      return asBuilder();
+    }
+
+    @Override
+    public HBaseTestAppender build() {
+      long size = FileSize.parse(maxSize, -1);
+      if (size <= 0) {
+        LOGGER.error("Invalid maxSize {}", size);
+      }
+      Layout<? extends Serializable> layout = getOrCreateLayout(StandardCharsets.UTF_8);
+      OutputStreamManager manager =
+        OutputStreamManager.getManager(target.name(), FACTORY, new FactoryData(target, layout));
+      return new HBaseTestAppender(getName(),
+        layout,
+        getFilter(),
+        isIgnoreExceptions(),
+        isImmediateFlush(),
+        getPropertyArray(),
+        manager,
+        size);
+    }
+  }
+
+  /**
+   * Data to pass to factory method.Unable to instantiate
+   */
+  private static class FactoryData {
+    private final Target target;
+    private final Layout<? extends Serializable> layout;
+
+    public FactoryData(Target target, Layout<? extends Serializable> layout) {
+      this.target = target;
+      this.layout = layout;
+    }
+  }
+
+  /**
+   * Factory to create the Appender.
+   */
+  private static class HBaseTestManagerFactory
+    implements ManagerFactory<HBaseTestOutputStreamManager, FactoryData> {
+
+    /**
+     * Create an OutputStreamManager.
+     * @param name The name of the entity to manage.
+     * @param data The data required to create the entity.
+     * @return The OutputStreamManager
+     */
+    @Override
+    public HBaseTestOutputStreamManager createManager(final String name, final FactoryData data) {
+      return new HBaseTestOutputStreamManager(data.target, data.layout);
+    }
+  }
+
+  @PluginBuilderFactory
+  public static <B extends Builder<B>> B newBuilder() {
+    return new Builder<B>().asBuilder();
+  }
+
+  private final long maxSize;
+
+  private final AtomicLong size = new AtomicLong(0);
+
+  private final AtomicBoolean stop = new AtomicBoolean(false);
+
+  private HBaseTestAppender(String name, Layout<? extends Serializable> layout, Filter filter,
+    boolean ignoreExceptions, boolean immediateFlush, Property[] properties,
+    OutputStreamManager manager, long maxSize) {
+    super(name, layout, filter, ignoreExceptions, immediateFlush, properties, manager);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void append(LogEvent event) {
+    if (stop.get()) {
+      return;
+    }
+    // for accounting, here we always convert the event to byte array first
+    // this will effect performance a bit but it is OK since this is for UT only
+    byte[] bytes = getLayout().toByteArray(event);
+    if (bytes == null || bytes.length == 0) {
+      return;
+    }
+    long sizeAfterAppend = size.addAndGet(bytes.length);
+    if (sizeAfterAppend >= maxSize) {
+      // stop logging if the log size exceeded the limit
+      if (stop.compareAndSet(false, true)) {
+        LOGGER.error("Log size exceeded the limit {}, will stop logging to prevent eating"
+          + " too much disk space", maxSize);
+      }
+      return;
+    }
+    super.append(event);
+  }
+}

Review Comment:
   Throw an Error out maybe? In general I'm not sure if this could work, because it could be a thread other than the main thread which enters here and exceeds the limit, fail this thread can not abort the test?



-- 
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@hbase.apache.org

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


[GitHub] [hbase] ndimiduk commented on a diff in pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on code in PR #4340:
URL: https://github.com/apache/hbase/pull/4340#discussion_r852832501


##########
hbase-logging/src/test/java/org/apache/hadoop/hbase/logging/HBaseTestAppender.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.hbase.logging;
+
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.ManagerFactory;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.apache.logging.log4j.core.appender.rolling.FileSize;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
+import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required;
+
+/**
+ * Log4j2 appender to be used when running UTs.
+ * <p/>
+ * The main point here is to limit the total output size to prevent eating all the space of our ci
+ * system when something wrong in our code.
+ * <p/>
+ * See HBASE-26947 for more details.
+ */
+@Plugin(name = HBaseTestAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+public final class HBaseTestAppender extends AbstractOutputStreamAppender<OutputStreamManager> {
+
+  public static final String PLUGIN_NAME = "HBaseTest";
+  private static final HBaseTestManagerFactory FACTORY = new HBaseTestManagerFactory();
+
+  public static class Builder<B extends Builder<B>> extends AbstractOutputStreamAppender.Builder<B>
+    implements org.apache.logging.log4j.core.util.Builder<HBaseTestAppender> {
+
+    @PluginBuilderAttribute
+    @Required
+    private Target target;
+
+    @PluginBuilderAttribute
+    @Required
+    private String maxSize;
+
+    public B setTarget(Target target) {
+      this.target = target;
+      return asBuilder();
+    }
+
+    public B setMaxSize(String maxSize) {
+      this.maxSize = maxSize;
+      return asBuilder();
+    }
+
+    @Override
+    public HBaseTestAppender build() {
+      long size = FileSize.parse(maxSize, -1);
+      if (size <= 0) {
+        LOGGER.error("Invalid maxSize {}", size);
+      }
+      Layout<? extends Serializable> layout = getOrCreateLayout(StandardCharsets.UTF_8);
+      OutputStreamManager manager =
+        OutputStreamManager.getManager(target.name(), FACTORY, new FactoryData(target, layout));
+      return new HBaseTestAppender(getName(),
+        layout,
+        getFilter(),
+        isIgnoreExceptions(),
+        isImmediateFlush(),
+        getPropertyArray(),
+        manager,
+        size);
+    }
+  }
+
+  /**
+   * Data to pass to factory method.Unable to instantiate
+   */
+  private static class FactoryData {
+    private final Target target;
+    private final Layout<? extends Serializable> layout;
+
+    public FactoryData(Target target, Layout<? extends Serializable> layout) {
+      this.target = target;
+      this.layout = layout;
+    }
+  }
+
+  /**
+   * Factory to create the Appender.
+   */
+  private static class HBaseTestManagerFactory
+    implements ManagerFactory<HBaseTestOutputStreamManager, FactoryData> {
+
+    /**
+     * Create an OutputStreamManager.
+     * @param name The name of the entity to manage.
+     * @param data The data required to create the entity.
+     * @return The OutputStreamManager
+     */
+    @Override
+    public HBaseTestOutputStreamManager createManager(final String name, final FactoryData data) {
+      return new HBaseTestOutputStreamManager(data.target, data.layout);
+    }
+  }
+
+  @PluginBuilderFactory
+  public static <B extends Builder<B>> B newBuilder() {
+    return new Builder<B>().asBuilder();
+  }
+
+  private final long maxSize;
+
+  private final AtomicLong size = new AtomicLong(0);
+
+  private final AtomicBoolean stop = new AtomicBoolean(false);
+
+  private HBaseTestAppender(String name, Layout<? extends Serializable> layout, Filter filter,
+    boolean ignoreExceptions, boolean immediateFlush, Property[] properties,
+    OutputStreamManager manager, long maxSize) {
+    super(name, layout, filter, ignoreExceptions, immediateFlush, properties, manager);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void append(LogEvent event) {
+    if (stop.get()) {
+      return;
+    }
+    // for accounting, here we always convert the event to byte array first
+    // this will effect performance a bit but it is OK since this is for UT only
+    byte[] bytes = getLayout().toByteArray(event);
+    if (bytes == null || bytes.length == 0) {
+      return;
+    }
+    long sizeAfterAppend = size.addAndGet(bytes.length);
+    if (sizeAfterAppend >= maxSize) {
+      // stop logging if the log size exceeded the limit
+      if (stop.compareAndSet(false, true)) {
+        LOGGER.error("Log size exceeded the limit {}, will stop logging to prevent eating"
+          + " too much disk space", maxSize);
+      }
+      return;
+    }
+    super.append(event);
+  }
+}

Review Comment:
   If asserting on the invariant is impossible, I'm okay with adding a comment here so that at least a future change to test execution will have a breadcrumb to follow.



-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache9 commented on a diff in pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4340:
URL: https://github.com/apache/hbase/pull/4340#discussion_r853300305


##########
hbase-logging/src/test/java/org/apache/hadoop/hbase/logging/HBaseTestAppender.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.hbase.logging;
+
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.ManagerFactory;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.apache.logging.log4j.core.appender.rolling.FileSize;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
+import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required;
+
+/**
+ * Log4j2 appender to be used when running UTs.
+ * <p/>
+ * The main point here is to limit the total output size to prevent eating all the space of our ci
+ * system when something wrong in our code.
+ * <p/>
+ * See HBASE-26947 for more details.
+ */
+@Plugin(name = HBaseTestAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+public final class HBaseTestAppender extends AbstractOutputStreamAppender<OutputStreamManager> {
+
+  public static final String PLUGIN_NAME = "HBaseTest";
+  private static final HBaseTestManagerFactory FACTORY = new HBaseTestManagerFactory();
+
+  public static class Builder<B extends Builder<B>> extends AbstractOutputStreamAppender.Builder<B>
+    implements org.apache.logging.log4j.core.util.Builder<HBaseTestAppender> {
+
+    @PluginBuilderAttribute
+    @Required
+    private Target target;
+
+    @PluginBuilderAttribute
+    @Required
+    private String maxSize;
+
+    public B setTarget(Target target) {
+      this.target = target;
+      return asBuilder();
+    }
+
+    public B setMaxSize(String maxSize) {
+      this.maxSize = maxSize;
+      return asBuilder();
+    }
+
+    @Override
+    public HBaseTestAppender build() {
+      long size = FileSize.parse(maxSize, -1);
+      if (size <= 0) {
+        LOGGER.error("Invalid maxSize {}", size);
+      }
+      Layout<? extends Serializable> layout = getOrCreateLayout(StandardCharsets.UTF_8);
+      OutputStreamManager manager =
+        OutputStreamManager.getManager(target.name(), FACTORY, new FactoryData(target, layout));
+      return new HBaseTestAppender(getName(),
+        layout,
+        getFilter(),
+        isIgnoreExceptions(),
+        isImmediateFlush(),
+        getPropertyArray(),
+        manager,
+        size);
+    }
+  }
+
+  /**
+   * Data to pass to factory method.Unable to instantiate
+   */
+  private static class FactoryData {
+    private final Target target;
+    private final Layout<? extends Serializable> layout;
+
+    public FactoryData(Target target, Layout<? extends Serializable> layout) {
+      this.target = target;
+      this.layout = layout;
+    }
+  }
+
+  /**
+   * Factory to create the Appender.
+   */
+  private static class HBaseTestManagerFactory
+    implements ManagerFactory<HBaseTestOutputStreamManager, FactoryData> {
+
+    /**
+     * Create an OutputStreamManager.
+     * @param name The name of the entity to manage.
+     * @param data The data required to create the entity.
+     * @return The OutputStreamManager
+     */
+    @Override
+    public HBaseTestOutputStreamManager createManager(final String name, final FactoryData data) {
+      return new HBaseTestOutputStreamManager(data.target, data.layout);
+    }
+  }
+
+  @PluginBuilderFactory
+  public static <B extends Builder<B>> B newBuilder() {
+    return new Builder<B>().asBuilder();
+  }
+
+  private final long maxSize;
+
+  private final AtomicLong size = new AtomicLong(0);
+
+  private final AtomicBoolean stop = new AtomicBoolean(false);
+
+  private HBaseTestAppender(String name, Layout<? extends Serializable> layout, Filter filter,
+    boolean ignoreExceptions, boolean immediateFlush, Property[] properties,
+    OutputStreamManager manager, long maxSize) {
+    super(name, layout, filter, ignoreExceptions, immediateFlush, properties, manager);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void append(LogEvent event) {
+    if (stop.get()) {
+      return;
+    }
+    // for accounting, here we always convert the event to byte array first
+    // this will effect performance a bit but it is OK since this is for UT only
+    byte[] bytes = getLayout().toByteArray(event);
+    if (bytes == null || bytes.length == 0) {
+      return;
+    }
+    long sizeAfterAppend = size.addAndGet(bytes.length);
+    if (sizeAfterAppend >= maxSize) {
+      // stop logging if the log size exceeded the limit
+      if (stop.compareAndSet(false, true)) {
+        LOGGER.error("Log size exceeded the limit {}, will stop logging to prevent eating"
+          + " too much disk space", maxSize);
+      }
+      return;
+    }
+    super.append(event);
+  }
+}

Review Comment:
   I've tried to use RunListener to reset the counter when starting a new test but failed... The behavior is really confusing, I mean when will the methods in the RunListener be called, and when will the surefire plugin rotate the output tests when reuseForks is true...
   
   So I just add some comments to pom.xml to say that why we can not set reuseForks to true.
   
   PTAL. @ndimiduk 



-- 
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@hbase.apache.org

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


[GitHub] [hbase] ndimiduk commented on a diff in pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on code in PR #4340:
URL: https://github.com/apache/hbase/pull/4340#discussion_r852831220


##########
hbase-logging/src/test/java/org/apache/hadoop/hbase/logging/HBaseTestAppender.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.hbase.logging;
+
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.ManagerFactory;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.apache.logging.log4j.core.appender.rolling.FileSize;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
+import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required;
+
+/**
+ * Log4j2 appender to be used when running UTs.
+ * <p/>
+ * The main point here is to limit the total output size to prevent eating all the space of our ci
+ * system when something wrong in our code.
+ * <p/>
+ * See HBASE-26947 for more details.
+ */
+@Plugin(name = HBaseTestAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+public final class HBaseTestAppender extends AbstractOutputStreamAppender<OutputStreamManager> {
+
+  public static final String PLUGIN_NAME = "HBaseTest";
+  private static final HBaseTestManagerFactory FACTORY = new HBaseTestManagerFactory();
+
+  public static class Builder<B extends Builder<B>> extends AbstractOutputStreamAppender.Builder<B>
+    implements org.apache.logging.log4j.core.util.Builder<HBaseTestAppender> {
+
+    @PluginBuilderAttribute
+    @Required
+    private Target target;
+
+    @PluginBuilderAttribute
+    @Required
+    private String maxSize;
+
+    public B setTarget(Target target) {
+      this.target = target;
+      return asBuilder();
+    }
+
+    public B setMaxSize(String maxSize) {
+      this.maxSize = maxSize;
+      return asBuilder();
+    }
+
+    @Override
+    public HBaseTestAppender build() {
+      long size = FileSize.parse(maxSize, -1);
+      if (size <= 0) {
+        LOGGER.error("Invalid maxSize {}", size);
+      }
+      Layout<? extends Serializable> layout = getOrCreateLayout(StandardCharsets.UTF_8);
+      OutputStreamManager manager =
+        OutputStreamManager.getManager(target.name(), FACTORY, new FactoryData(target, layout));
+      return new HBaseTestAppender(getName(),
+        layout,
+        getFilter(),
+        isIgnoreExceptions(),
+        isImmediateFlush(),
+        getPropertyArray(),
+        manager,
+        size);
+    }
+  }
+
+  /**
+   * Data to pass to factory method.Unable to instantiate
+   */
+  private static class FactoryData {
+    private final Target target;
+    private final Layout<? extends Serializable> layout;
+
+    public FactoryData(Target target, Layout<? extends Serializable> layout) {
+      this.target = target;
+      this.layout = layout;
+    }
+  }
+
+  /**
+   * Factory to create the Appender.
+   */
+  private static class HBaseTestManagerFactory
+    implements ManagerFactory<HBaseTestOutputStreamManager, FactoryData> {
+
+    /**
+     * Create an OutputStreamManager.
+     * @param name The name of the entity to manage.
+     * @param data The data required to create the entity.
+     * @return The OutputStreamManager
+     */
+    @Override
+    public HBaseTestOutputStreamManager createManager(final String name, final FactoryData data) {
+      return new HBaseTestOutputStreamManager(data.target, data.layout);
+    }
+  }
+
+  @PluginBuilderFactory
+  public static <B extends Builder<B>> B newBuilder() {
+    return new Builder<B>().asBuilder();
+  }
+
+  private final long maxSize;
+
+  private final AtomicLong size = new AtomicLong(0);
+
+  private final AtomicBoolean stop = new AtomicBoolean(false);
+
+  private HBaseTestAppender(String name, Layout<? extends Serializable> layout, Filter filter,
+    boolean ignoreExceptions, boolean immediateFlush, Property[] properties,
+    OutputStreamManager manager, long maxSize) {
+    super(name, layout, filter, ignoreExceptions, immediateFlush, properties, manager);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void append(LogEvent event) {
+    if (stop.get()) {
+      return;
+    }
+    // for accounting, here we always convert the event to byte array first
+    // this will effect performance a bit but it is OK since this is for UT only
+    byte[] bytes = getLayout().toByteArray(event);
+    if (bytes == null || bytes.length == 0) {
+      return;
+    }
+    long sizeAfterAppend = size.addAndGet(bytes.length);
+    if (sizeAfterAppend >= maxSize) {
+      // stop logging if the log size exceeded the limit
+      if (stop.compareAndSet(false, true)) {
+        LOGGER.error("Log size exceeded the limit {}, will stop logging to prevent eating"
+          + " too much disk space", maxSize);
+      }
+      return;
+    }
+    super.append(event);
+  }
+}

Review Comment:
   > We do not use reuseForks so this is not a problem. Every test will have its own process.
   
   In that case, it would be better to add an assertion that this invariant is held. With that, I'm +1.



-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1103242519

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 41s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 19s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 24s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 55s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m  5s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 52s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 23s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 324m 35s |  root in the patch failed.  |
   |  |   | 346m 59s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0d6050b246d1 5.4.0-1025-aws #25~18.04.1-Ubuntu SMP Fri Sep 11 12:03:04 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 35aa57e445 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/testReport/ |
   | Max. process+thread count | 2291 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache9 commented on a diff in pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4340:
URL: https://github.com/apache/hbase/pull/4340#discussion_r848951037


##########
hbase-logging/src/test/java/org/apache/hadoop/hbase/logging/HBaseTestAppender.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.hbase.logging;
+
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.ManagerFactory;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.apache.logging.log4j.core.appender.rolling.FileSize;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
+import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required;
+
+/**
+ * Log4j2 appender to be used when running UTs.
+ * <p/>
+ * The main point here is to limit the total output size to prevent eating all the space of our ci
+ * system when something wrong in our code.
+ * <p/>
+ * See HBASE-26947 for more details.
+ */
+@Plugin(name = HBaseTestAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+public final class HBaseTestAppender extends AbstractOutputStreamAppender<OutputStreamManager> {
+
+  public static final String PLUGIN_NAME = "HBaseTest";
+  private static final HBaseTestManagerFactory FACTORY = new HBaseTestManagerFactory();
+
+  public static class Builder<B extends Builder<B>> extends AbstractOutputStreamAppender.Builder<B>
+    implements org.apache.logging.log4j.core.util.Builder<HBaseTestAppender> {
+
+    @PluginBuilderAttribute
+    @Required
+    private Target target;
+
+    @PluginBuilderAttribute
+    @Required
+    private String maxSize;
+
+    public B setTarget(Target target) {
+      this.target = target;
+      return asBuilder();
+    }
+
+    public B setMaxSize(String maxSize) {
+      this.maxSize = maxSize;
+      return asBuilder();
+    }
+
+    @Override
+    public HBaseTestAppender build() {
+      long size = FileSize.parse(maxSize, -1);
+      if (size <= 0) {
+        LOGGER.error("Invalid maxSize {}", size);
+      }
+      Layout<? extends Serializable> layout = getOrCreateLayout(StandardCharsets.UTF_8);
+      OutputStreamManager manager =
+        OutputStreamManager.getManager(target.name(), FACTORY, new FactoryData(target, layout));
+      return new HBaseTestAppender(getName(),
+        layout,
+        getFilter(),
+        isIgnoreExceptions(),
+        isImmediateFlush(),
+        getPropertyArray(),
+        manager,
+        size);
+    }
+  }
+
+  /**
+   * Data to pass to factory method.Unable to instantiate
+   */
+  private static class FactoryData {
+    private final Target target;
+    private final Layout<? extends Serializable> layout;
+
+    public FactoryData(Target target, Layout<? extends Serializable> layout) {
+      this.target = target;
+      this.layout = layout;
+    }
+  }
+
+  /**
+   * Factory to create the Appender.
+   */
+  private static class HBaseTestManagerFactory
+    implements ManagerFactory<HBaseTestOutputStreamManager, FactoryData> {
+
+    /**
+     * Create an OutputStreamManager.
+     * @param name The name of the entity to manage.
+     * @param data The data required to create the entity.
+     * @return The OutputStreamManager
+     */
+    @Override
+    public HBaseTestOutputStreamManager createManager(final String name, final FactoryData data) {
+      return new HBaseTestOutputStreamManager(data.target, data.layout);
+    }
+  }
+
+  @PluginBuilderFactory
+  public static <B extends Builder<B>> B newBuilder() {
+    return new Builder<B>().asBuilder();
+  }
+
+  private final long maxSize;
+
+  private final AtomicLong size = new AtomicLong(0);
+
+  private final AtomicBoolean stop = new AtomicBoolean(false);
+
+  private HBaseTestAppender(String name, Layout<? extends Serializable> layout, Filter filter,
+    boolean ignoreExceptions, boolean immediateFlush, Property[] properties,
+    OutputStreamManager manager, long maxSize) {
+    super(name, layout, filter, ignoreExceptions, immediateFlush, properties, manager);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void append(LogEvent event) {
+    if (stop.get()) {
+      return;
+    }
+    // for accounting, here we always convert the event to byte array first
+    // this will effect performance a bit but it is OK since this is for UT only
+    byte[] bytes = getLayout().toByteArray(event);
+    if (bytes == null || bytes.length == 0) {
+      return;
+    }
+    long sizeAfterAppend = size.addAndGet(bytes.length);
+    if (sizeAfterAppend >= maxSize) {
+      // stop logging if the log size exceeded the limit
+      if (stop.compareAndSet(false, true)) {
+        LOGGER.error("Log size exceeded the limit {}, will stop logging to prevent eating"
+          + " too much disk space", maxSize);
+      }
+      return;
+    }
+    super.append(event);
+  }
+}

Review Comment:
   We do not use reuseForks so this is not a problem. Every test will have its own process.



-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1098698471

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m  3s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m  0s |  master passed  |
   | +1 :green_heart: |  compile  |  15m  1s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m  7s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  16m 54s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 16s |  the patch passed  |
   | +1 :green_heart: |  compile  |  17m 47s |  the patch passed  |
   | -0 :warning: |  javac  |  17m 47s |  root generated 1 new + 1472 unchanged - 0 fixed = 1473 total (was 1472)  |
   | +1 :green_heart: |  checkstyle  |   2m 19s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  1s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  hadoopcheck  |  25m 24s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |  20m 20s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   | 123m 37s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux 2f12901f18d9 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / f990f56e8e |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/artifact/yetus-general-check/output/diff-compile-javac-root.txt |
   | Max. process+thread count | 137 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] ndimiduk commented on a diff in pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on code in PR #4340:
URL: https://github.com/apache/hbase/pull/4340#discussion_r848773944


##########
hbase-logging/src/test/java/org/apache/hadoop/hbase/logging/HBaseTestAppender.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.hbase.logging;
+
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.ManagerFactory;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.apache.logging.log4j.core.appender.rolling.FileSize;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
+import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required;
+
+/**
+ * Log4j2 appender to be used when running UTs.
+ * <p/>
+ * The main point here is to limit the total output size to prevent eating all the space of our ci
+ * system when something wrong in our code.
+ * <p/>
+ * See HBASE-26947 for more details.
+ */
+@Plugin(name = HBaseTestAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+public final class HBaseTestAppender extends AbstractOutputStreamAppender<OutputStreamManager> {
+
+  public static final String PLUGIN_NAME = "HBaseTest";
+  private static final HBaseTestManagerFactory FACTORY = new HBaseTestManagerFactory();
+
+  public static class Builder<B extends Builder<B>> extends AbstractOutputStreamAppender.Builder<B>
+    implements org.apache.logging.log4j.core.util.Builder<HBaseTestAppender> {
+
+    @PluginBuilderAttribute
+    @Required
+    private Target target;
+
+    @PluginBuilderAttribute
+    @Required
+    private String maxSize;
+
+    public B setTarget(Target target) {
+      this.target = target;
+      return asBuilder();
+    }
+
+    public B setMaxSize(String maxSize) {
+      this.maxSize = maxSize;
+      return asBuilder();
+    }
+
+    @Override
+    public HBaseTestAppender build() {
+      long size = FileSize.parse(maxSize, -1);
+      if (size <= 0) {
+        LOGGER.error("Invalid maxSize {}", size);
+      }
+      Layout<? extends Serializable> layout = getOrCreateLayout(StandardCharsets.UTF_8);
+      OutputStreamManager manager =
+        OutputStreamManager.getManager(target.name(), FACTORY, new FactoryData(target, layout));
+      return new HBaseTestAppender(getName(),
+        layout,
+        getFilter(),
+        isIgnoreExceptions(),
+        isImmediateFlush(),
+        getPropertyArray(),
+        manager,
+        size);
+    }
+  }
+
+  /**
+   * Data to pass to factory method.Unable to instantiate
+   */
+  private static class FactoryData {
+    private final Target target;
+    private final Layout<? extends Serializable> layout;
+
+    public FactoryData(Target target, Layout<? extends Serializable> layout) {
+      this.target = target;
+      this.layout = layout;
+    }
+  }
+
+  /**
+   * Factory to create the Appender.
+   */
+  private static class HBaseTestManagerFactory
+    implements ManagerFactory<HBaseTestOutputStreamManager, FactoryData> {
+
+    /**
+     * Create an OutputStreamManager.
+     * @param name The name of the entity to manage.
+     * @param data The data required to create the entity.
+     * @return The OutputStreamManager
+     */
+    @Override
+    public HBaseTestOutputStreamManager createManager(final String name, final FactoryData data) {
+      return new HBaseTestOutputStreamManager(data.target, data.layout);
+    }
+  }
+
+  @PluginBuilderFactory
+  public static <B extends Builder<B>> B newBuilder() {
+    return new Builder<B>().asBuilder();
+  }
+
+  private final long maxSize;
+
+  private final AtomicLong size = new AtomicLong(0);
+
+  private final AtomicBoolean stop = new AtomicBoolean(false);
+
+  private HBaseTestAppender(String name, Layout<? extends Serializable> layout, Filter filter,
+    boolean ignoreExceptions, boolean immediateFlush, Property[] properties,
+    OutputStreamManager manager, long maxSize) {
+    super(name, layout, filter, ignoreExceptions, immediateFlush, properties, manager);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void append(LogEvent event) {
+    if (stop.get()) {
+      return;
+    }
+    // for accounting, here we always convert the event to byte array first
+    // this will effect performance a bit but it is OK since this is for UT only
+    byte[] bytes = getLayout().toByteArray(event);
+    if (bytes == null || bytes.length == 0) {
+      return;
+    }
+    long sizeAfterAppend = size.addAndGet(bytes.length);
+    if (sizeAfterAppend >= maxSize) {
+      // stop logging if the log size exceeded the limit
+      if (stop.compareAndSet(false, true)) {
+        LOGGER.error("Log size exceeded the limit {}, will stop logging to prevent eating"
+          + " too much disk space", maxSize);
+      }
+      return;
+    }
+    super.append(event);
+  }
+}

Review Comment:
   Or maybe it's one log line per Event object. I see.
   
   I'm fine with this. I think it's better to let the test finish than to kill the build when the log limit is exceeded. In my experience, it's relatively rare that logs from CI are used by anyone. This strategy for logging once max log volume for a test is exceeded is fine by me.
   
   What happens when the appender is for some class that's used over and over in test? There's no way to reset this counter to be a per-test-class accumulator, right?



-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache9 commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1100112823

   Any other concerns? @ndimiduk @busbey 


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1098957828

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 54s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m  2s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 51s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m  2s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 30s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 44s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  4s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  4s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 42s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m  9s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 403m 30s |  root in the patch failed.  |
   |  |   | 449m  3s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3e839e8bc026 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / f990f56e8e |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/testReport/ |
   | Max. process+thread count | 2295 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache9 commented on a diff in pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on code in PR #4340:
URL: https://github.com/apache/hbase/pull/4340#discussion_r852845672


##########
hbase-logging/src/test/java/org/apache/hadoop/hbase/logging/HBaseTestAppender.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.hbase.logging;
+
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.ManagerFactory;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.apache.logging.log4j.core.appender.rolling.FileSize;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
+import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required;
+
+/**
+ * Log4j2 appender to be used when running UTs.
+ * <p/>
+ * The main point here is to limit the total output size to prevent eating all the space of our ci
+ * system when something wrong in our code.
+ * <p/>
+ * See HBASE-26947 for more details.
+ */
+@Plugin(name = HBaseTestAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+public final class HBaseTestAppender extends AbstractOutputStreamAppender<OutputStreamManager> {
+
+  public static final String PLUGIN_NAME = "HBaseTest";
+  private static final HBaseTestManagerFactory FACTORY = new HBaseTestManagerFactory();
+
+  public static class Builder<B extends Builder<B>> extends AbstractOutputStreamAppender.Builder<B>
+    implements org.apache.logging.log4j.core.util.Builder<HBaseTestAppender> {
+
+    @PluginBuilderAttribute
+    @Required
+    private Target target;
+
+    @PluginBuilderAttribute
+    @Required
+    private String maxSize;
+
+    public B setTarget(Target target) {
+      this.target = target;
+      return asBuilder();
+    }
+
+    public B setMaxSize(String maxSize) {
+      this.maxSize = maxSize;
+      return asBuilder();
+    }
+
+    @Override
+    public HBaseTestAppender build() {
+      long size = FileSize.parse(maxSize, -1);
+      if (size <= 0) {
+        LOGGER.error("Invalid maxSize {}", size);
+      }
+      Layout<? extends Serializable> layout = getOrCreateLayout(StandardCharsets.UTF_8);
+      OutputStreamManager manager =
+        OutputStreamManager.getManager(target.name(), FACTORY, new FactoryData(target, layout));
+      return new HBaseTestAppender(getName(),
+        layout,
+        getFilter(),
+        isIgnoreExceptions(),
+        isImmediateFlush(),
+        getPropertyArray(),
+        manager,
+        size);
+    }
+  }
+
+  /**
+   * Data to pass to factory method.Unable to instantiate
+   */
+  private static class FactoryData {
+    private final Target target;
+    private final Layout<? extends Serializable> layout;
+
+    public FactoryData(Target target, Layout<? extends Serializable> layout) {
+      this.target = target;
+      this.layout = layout;
+    }
+  }
+
+  /**
+   * Factory to create the Appender.
+   */
+  private static class HBaseTestManagerFactory
+    implements ManagerFactory<HBaseTestOutputStreamManager, FactoryData> {
+
+    /**
+     * Create an OutputStreamManager.
+     * @param name The name of the entity to manage.
+     * @param data The data required to create the entity.
+     * @return The OutputStreamManager
+     */
+    @Override
+    public HBaseTestOutputStreamManager createManager(final String name, final FactoryData data) {
+      return new HBaseTestOutputStreamManager(data.target, data.layout);
+    }
+  }
+
+  @PluginBuilderFactory
+  public static <B extends Builder<B>> B newBuilder() {
+    return new Builder<B>().asBuilder();
+  }
+
+  private final long maxSize;
+
+  private final AtomicLong size = new AtomicLong(0);
+
+  private final AtomicBoolean stop = new AtomicBoolean(false);
+
+  private HBaseTestAppender(String name, Layout<? extends Serializable> layout, Filter filter,
+    boolean ignoreExceptions, boolean immediateFlush, Property[] properties,
+    OutputStreamManager manager, long maxSize) {
+    super(name, layout, filter, ignoreExceptions, immediateFlush, properties, manager);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void append(LogEvent event) {
+    if (stop.get()) {
+      return;
+    }
+    // for accounting, here we always convert the event to byte array first
+    // this will effect performance a bit but it is OK since this is for UT only
+    byte[] bytes = getLayout().toByteArray(event);
+    if (bytes == null || bytes.length == 0) {
+      return;
+    }
+    long sizeAfterAppend = size.addAndGet(bytes.length);
+    if (sizeAfterAppend >= maxSize) {
+      // stop logging if the log size exceeded the limit
+      if (stop.compareAndSet(false, true)) {
+        LOGGER.error("Log size exceeded the limit {}, will stop logging to prevent eating"
+          + " too much disk space", maxSize);
+      }
+      return;
+    }
+    super.append(event);
+  }
+}

Review Comment:
   I do not know how to add the assertion in code to test a pom config :(
   But anyway, we could add a comment about this, both in the appender code and also in pom.



-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1103187150

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 22s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 41s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 47s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 52s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 20s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 41s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 41s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 44s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 49s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 225m 52s |  root in the patch passed.  |
   |  |   | 250m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 646b70bdcf37 5.4.0-1071-aws #76~18.04.1-Ubuntu SMP Mon Mar 28 17:49:57 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 35aa57e445 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/testReport/ |
   | Max. process+thread count | 4862 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1096858135

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  5s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 28s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 10s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 46s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 11s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 38s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 10s |  hbase-logging in the patch failed.  |
   | -0 :warning: |  javac  |   0m 10s |  hbase-logging in the patch failed.  |
   | -1 :x: |  shadedjars  |   0m 37s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 10s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m  9s |  hbase-logging in the patch failed.  |
   |  |   |  10m 22s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 4a3f36a43101 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea9bc92ce2 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | mvninstall | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk8-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-logging.txt |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-logging.txt |
   | shadedjars | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-logging.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/testReport/ |
   | Max. process+thread count | 62 (vs. ulimit of 30000) |
   | modules | C: hbase-logging U: hbase-logging |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache9 commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1096829643

   It is not easy to write a UT for the log4j2 appender so I just test it manually.
   
   Change the 'appender.console.maxSize' property in log4j2.properties to a very small value, for example, 1K, you will soon see this output when running UT in IDE
   
   ```
   2022-04-12 22:44:20,366 Time-limited test ERROR Log size exceeded the limit 1024, will stop logging to prevent eating too much disk space
   ```
   
   So I think this approach works.
   
   And I set default value to 1G, which should be enough for most cases...
   
   @busbey @ndimiduk FYI. I think this could solve the recent space problem of our ci system.


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache9 commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache9 commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1103289726

   Ping @ndimiduk 


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1098777403

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  1s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 38s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 53s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 44s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  7s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 33s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 53s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 53s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 42s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m  6s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 255m 15s |  root in the patch passed.  |
   |  |   | 279m 23s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3fed9f2fca07 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / f990f56e8e |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/testReport/ |
   | Max. process+thread count | 5689 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/4/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1097594351

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  1s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 57s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 40s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 23s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 44s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 42s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 42s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 22s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 55s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 359m  0s |  root in the patch failed.  |
   |  |   | 387m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 4298147e3939 5.4.0-1025-aws #25~18.04.1-Ubuntu SMP Fri Sep 11 12:03:04 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea9bc92ce2 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/testReport/ |
   | Max. process+thread count | 2606 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] busbey commented on a diff in pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
busbey commented on code in PR #4340:
URL: https://github.com/apache/hbase/pull/4340#discussion_r848597633


##########
hbase-logging/src/test/java/org/apache/hadoop/hbase/logging/HBaseTestAppender.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.hbase.logging;
+
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.ManagerFactory;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.apache.logging.log4j.core.appender.rolling.FileSize;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
+import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required;
+
+/**
+ * Log4j2 appender to be used when running UTs.
+ * <p/>
+ * The main point here is to limit the total output size to prevent eating all the space of our ci
+ * system when something wrong in our code.
+ * <p/>
+ * See HBASE-26947 for more details.
+ */
+@Plugin(name = HBaseTestAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+public final class HBaseTestAppender extends AbstractOutputStreamAppender<OutputStreamManager> {
+
+  public static final String PLUGIN_NAME = "HBaseTest";
+  private static final HBaseTestManagerFactory FACTORY = new HBaseTestManagerFactory();
+
+  public static class Builder<B extends Builder<B>> extends AbstractOutputStreamAppender.Builder<B>
+    implements org.apache.logging.log4j.core.util.Builder<HBaseTestAppender> {
+
+    @PluginBuilderAttribute
+    @Required
+    private Target target;
+
+    @PluginBuilderAttribute
+    @Required
+    private String maxSize;
+
+    public B setTarget(Target target) {
+      this.target = target;
+      return asBuilder();
+    }
+
+    public B setMaxSize(String maxSize) {
+      this.maxSize = maxSize;
+      return asBuilder();
+    }
+
+    @Override
+    public HBaseTestAppender build() {
+      long size = FileSize.parse(maxSize, -1);
+      if (size <= 0) {
+        LOGGER.error("Invalid maxSize {}", size);
+      }
+      Layout<? extends Serializable> layout = getOrCreateLayout(StandardCharsets.UTF_8);
+      OutputStreamManager manager =
+        OutputStreamManager.getManager(target.name(), FACTORY, new FactoryData(target, layout));
+      return new HBaseTestAppender(getName(),
+        layout,
+        getFilter(),
+        isIgnoreExceptions(),
+        isImmediateFlush(),
+        getPropertyArray(),
+        manager,
+        size);
+    }
+  }
+
+  /**
+   * Data to pass to factory method.Unable to instantiate
+   */
+  private static class FactoryData {
+    private final Target target;
+    private final Layout<? extends Serializable> layout;
+
+    public FactoryData(Target target, Layout<? extends Serializable> layout) {
+      this.target = target;
+      this.layout = layout;
+    }
+  }
+
+  /**
+   * Factory to create the Appender.
+   */
+  private static class HBaseTestManagerFactory
+    implements ManagerFactory<HBaseTestOutputStreamManager, FactoryData> {
+
+    /**
+     * Create an OutputStreamManager.
+     * @param name The name of the entity to manage.
+     * @param data The data required to create the entity.
+     * @return The OutputStreamManager
+     */
+    @Override
+    public HBaseTestOutputStreamManager createManager(final String name, final FactoryData data) {
+      return new HBaseTestOutputStreamManager(data.target, data.layout);
+    }
+  }
+
+  @PluginBuilderFactory
+  public static <B extends Builder<B>> B newBuilder() {
+    return new Builder<B>().asBuilder();
+  }
+
+  private final long maxSize;
+
+  private final AtomicLong size = new AtomicLong(0);
+
+  private final AtomicBoolean stop = new AtomicBoolean(false);
+
+  private HBaseTestAppender(String name, Layout<? extends Serializable> layout, Filter filter,
+    boolean ignoreExceptions, boolean immediateFlush, Property[] properties,
+    OutputStreamManager manager, long maxSize) {
+    super(name, layout, filter, ignoreExceptions, immediateFlush, properties, manager);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void append(LogEvent event) {
+    if (stop.get()) {
+      return;
+    }
+    // for accounting, here we always convert the event to byte array first
+    // this will effect performance a bit but it is OK since this is for UT only
+    byte[] bytes = getLayout().toByteArray(event);
+    if (bytes == null || bytes.length == 0) {
+      return;
+    }
+    long sizeAfterAppend = size.addAndGet(bytes.length);
+    if (sizeAfterAppend >= maxSize) {
+      // stop logging if the log size exceeded the limit
+      if (stop.compareAndSet(false, true)) {
+        LOGGER.error("Log size exceeded the limit {}, will stop logging to prevent eating"
+          + " too much disk space", maxSize);
+      }
+      return;
+    }
+    super.append(event);
+  }
+}

Review Comment:
   Can we abort the test in this case rather than having the test continue without logs?



-- 
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@hbase.apache.org

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


[GitHub] [hbase] ndimiduk commented on a diff in pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on code in PR #4340:
URL: https://github.com/apache/hbase/pull/4340#discussion_r848769758


##########
hbase-logging/src/test/java/org/apache/hadoop/hbase/logging/HBaseTestAppender.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop.hbase.logging;
+
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.Core;
+import org.apache.logging.log4j.core.Filter;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.appender.AbstractOutputStreamAppender;
+import org.apache.logging.log4j.core.appender.ManagerFactory;
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+import org.apache.logging.log4j.core.appender.rolling.FileSize;
+import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
+import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required;
+
+/**
+ * Log4j2 appender to be used when running UTs.
+ * <p/>
+ * The main point here is to limit the total output size to prevent eating all the space of our ci
+ * system when something wrong in our code.
+ * <p/>
+ * See HBASE-26947 for more details.
+ */
+@Plugin(name = HBaseTestAppender.PLUGIN_NAME, category = Core.CATEGORY_NAME,
+    elementType = Appender.ELEMENT_TYPE, printObject = true)
+public final class HBaseTestAppender extends AbstractOutputStreamAppender<OutputStreamManager> {
+
+  public static final String PLUGIN_NAME = "HBaseTest";
+  private static final HBaseTestManagerFactory FACTORY = new HBaseTestManagerFactory();
+
+  public static class Builder<B extends Builder<B>> extends AbstractOutputStreamAppender.Builder<B>
+    implements org.apache.logging.log4j.core.util.Builder<HBaseTestAppender> {
+
+    @PluginBuilderAttribute
+    @Required
+    private Target target;
+
+    @PluginBuilderAttribute
+    @Required
+    private String maxSize;
+
+    public B setTarget(Target target) {
+      this.target = target;
+      return asBuilder();
+    }
+
+    public B setMaxSize(String maxSize) {
+      this.maxSize = maxSize;
+      return asBuilder();
+    }
+
+    @Override
+    public HBaseTestAppender build() {
+      long size = FileSize.parse(maxSize, -1);
+      if (size <= 0) {
+        LOGGER.error("Invalid maxSize {}", size);
+      }
+      Layout<? extends Serializable> layout = getOrCreateLayout(StandardCharsets.UTF_8);
+      OutputStreamManager manager =
+        OutputStreamManager.getManager(target.name(), FACTORY, new FactoryData(target, layout));
+      return new HBaseTestAppender(getName(),
+        layout,
+        getFilter(),
+        isIgnoreExceptions(),
+        isImmediateFlush(),
+        getPropertyArray(),
+        manager,
+        size);
+    }
+  }
+
+  /**
+   * Data to pass to factory method.Unable to instantiate
+   */
+  private static class FactoryData {
+    private final Target target;
+    private final Layout<? extends Serializable> layout;
+
+    public FactoryData(Target target, Layout<? extends Serializable> layout) {
+      this.target = target;
+      this.layout = layout;
+    }
+  }
+
+  /**
+   * Factory to create the Appender.
+   */
+  private static class HBaseTestManagerFactory
+    implements ManagerFactory<HBaseTestOutputStreamManager, FactoryData> {
+
+    /**
+     * Create an OutputStreamManager.
+     * @param name The name of the entity to manage.
+     * @param data The data required to create the entity.
+     * @return The OutputStreamManager
+     */
+    @Override
+    public HBaseTestOutputStreamManager createManager(final String name, final FactoryData data) {
+      return new HBaseTestOutputStreamManager(data.target, data.layout);
+    }
+  }
+
+  @PluginBuilderFactory
+  public static <B extends Builder<B>> B newBuilder() {
+    return new Builder<B>().asBuilder();
+  }
+
+  private final long maxSize;
+
+  private final AtomicLong size = new AtomicLong(0);
+
+  private final AtomicBoolean stop = new AtomicBoolean(false);
+
+  private HBaseTestAppender(String name, Layout<? extends Serializable> layout, Filter filter,
+    boolean ignoreExceptions, boolean immediateFlush, Property[] properties,
+    OutputStreamManager manager, long maxSize) {
+    super(name, layout, filter, ignoreExceptions, immediateFlush, properties, manager);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  public void append(LogEvent event) {
+    if (stop.get()) {
+      return;
+    }
+    // for accounting, here we always convert the event to byte array first
+    // this will effect performance a bit but it is OK since this is for UT only
+    byte[] bytes = getLayout().toByteArray(event);
+    if (bytes == null || bytes.length == 0) {
+      return;
+    }
+    long sizeAfterAppend = size.addAndGet(bytes.length);
+    if (sizeAfterAppend >= maxSize) {
+      // stop logging if the log size exceeded the limit
+      if (stop.compareAndSet(false, true)) {
+        LOGGER.error("Log size exceeded the limit {}, will stop logging to prevent eating"
+          + " too much disk space", maxSize);
+      }
+      return;
+    }
+    super.append(event);
+  }
+}

Review Comment:
   I've done this in another logging framework but not log4j2. This is appending events to a single log line, building it up, is that correct? I think it's fine if we truncate log lines at a certain length, no need to error or whatever, just terminate the message. I'm a little surprised that this not available to use as a configuration option.
   
   In fact, the pattern layout does support it. https://stackoverflow.com/questions/27502536/limit-max-message-size-in-log4j2-pattern



-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache9 merged pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache9 merged PR #4340:
URL: https://github.com/apache/hbase/pull/4340


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1096863012

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 34s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 44s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 10s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 25s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 12s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 52s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m  9s |  hbase-logging in the patch failed.  |
   | -0 :warning: |  javac  |   0m  9s |  hbase-logging in the patch failed.  |
   | -1 :x: |  shadedjars  |   0m 54s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m  9s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m  9s |  hbase-logging in the patch failed.  |
   |  |   |  14m 17s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c6bd910e018d 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea9bc92ce2 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | mvninstall | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk11-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-logging.txt |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-logging.txt |
   | shadedjars | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-logging.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/testReport/ |
   | Max. process+thread count | 72 (vs. ulimit of 30000) |
   | modules | C: hbase-logging U: hbase-logging |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1097426506

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 13s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 12s |  master passed  |
   | +1 :green_heart: |  compile  |   6m 15s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m  9s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   7m 29s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m  7s |  the patch passed  |
   | +1 :green_heart: |  compile  |   6m  9s |  the patch passed  |
   | -0 :warning: |  javac  |   6m  9s |  root generated 1 new + 1472 unchanged - 0 fixed = 1473 total (was 1472)  |
   | +1 :green_heart: |  checkstyle  |   1m  9s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  1s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 29s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |   7m 40s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 20s |  The patch does not generate ASF License warnings.  |
   |  |   |  52m 41s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux 67dc595b8ba5 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea9bc92ce2 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/artifact/yetus-general-check/output/diff-compile-javac-root.txt |
   | Max. process+thread count | 138 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1097583081

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 32s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 51s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 10s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 29s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 20s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 331m 26s |  root in the patch passed.  |
   |  |   | 367m 29s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c50880a3c233 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea9bc92ce2 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/testReport/ |
   | Max. process+thread count | 4903 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/3/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1102999953

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 44s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 37s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 16s |  master passed  |
   | +1 :green_heart: |  compile  |  16m 41s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 34s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  19m 12s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 54s |  the patch passed  |
   | +1 :green_heart: |  compile  |  17m 24s |  the patch passed  |
   | -0 :warning: |  javac  |  17m 24s |  root generated 1 new + 1472 unchanged - 0 fixed = 1473 total (was 1472)  |
   | +1 :green_heart: |  checkstyle  |   2m 24s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  hadoopcheck  |  25m 45s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |  18m 12s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 38s |  The patch does not generate ASF License warnings.  |
   |  |   | 125m 27s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux 3f2d86ccdcc6 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 35aa57e445 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/artifact/yetus-general-check/output/diff-compile-javac-root.txt |
   | Max. process+thread count | 138 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/5/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1097254098

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 21s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 34s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   3m 38s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 38s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 14s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 33s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 33s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   3m 40s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   1m 27s |  root generated 4 new + 81 unchanged - 4 fixed = 85 total (was 85)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 340m 51s |  root in the patch passed.  |
   |  |   | 363m 27s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux bcc2266b3b60 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea9bc92ce2 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/testReport/ |
   | Max. process+thread count | 4543 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1096964761

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 12s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   8m 52s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 20s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  11m 23s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 46s |  the patch passed  |
   | +1 :green_heart: |  compile  |  10m 14s |  the patch passed  |
   | -0 :warning: |  javac  |  10m 14s |  root generated 5 new + 1467 unchanged - 5 fixed = 1472 total (was 1472)  |
   | +1 :green_heart: |  checkstyle  |   1m 58s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 49s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |   7m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 20s |  The patch does not generate ASF License warnings.  |
   |  |   |  68m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux 10e58bfdc1ca 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea9bc92ce2 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/artifact/yetus-general-check/output/diff-compile-javac-root.txt |
   | Max. process+thread count | 140 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1096854906

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 27s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 22s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 14s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m  8s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   0m 18s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 37s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m  6s |  hbase-logging in the patch failed.  |
   | -0 :warning: |  javac  |   0m  6s |  hbase-logging in the patch failed.  |
   | +1 :green_heart: |  checkstyle  |   0m  5s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | -1 :x: |  hadoopcheck  |   0m  6s |  The patch causes 10 errors with Hadoop v3.1.2.  |
   | -1 :x: |  hadoopcheck  |   0m 12s |  The patch causes 10 errors with Hadoop v3.2.2.  |
   | -1 :x: |  hadoopcheck  |   0m 19s |  The patch causes 10 errors with Hadoop v3.3.1.  |
   | -1 :x: |  spotbugs  |   0m  6s |  hbase-logging in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  9s |  The patch does not generate ASF License warnings.  |
   |  |   |   7m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux ed9d5fe97259 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea9bc92ce2 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | mvninstall | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-general-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-general-check/output/patch-compile-hbase-logging.txt |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-general-check/output/patch-compile-hbase-logging.txt |
   | hadoopcheck | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-general-check/output/patch-javac-3.1.2.txt |
   | hadoopcheck | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-general-check/output/patch-javac-3.2.2.txt |
   | hadoopcheck | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-general-check/output/patch-javac-3.3.1.txt |
   | spotbugs | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/artifact/yetus-general-check/output/patch-spotbugs-hbase-logging.txt |
   | Max. process+thread count | 60 (vs. ulimit of 30000) |
   | modules | C: hbase-logging U: hbase-logging |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/1/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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


[GitHub] [hbase] Apache-HBase commented on pull request #4340: HBASE-26947 Implement a special TestAppender to limit the size of tes…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on PR #4340:
URL: https://github.com/apache/hbase/pull/4340#issuecomment-1096948363

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 38s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  5s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 43s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  3s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 19s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 55s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  2s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m  7s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  16m 58s |  root in the patch failed.  |
   |  |   |  50m 53s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/4340 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 7927cfad6e99 5.4.0-90-generic #101-Ubuntu SMP Fri Oct 15 20:00:55 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ea9bc92ce2 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/testReport/ |
   | Max. process+thread count | 816 (vs. ulimit of 30000) |
   | modules | C: hbase-logging . U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-4340/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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@hbase.apache.org

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