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 2020/09/28 20:10:57 UTC

[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r496194646



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;

Review comment:
       I chose to make the field of a concrete type because we depend on the semantics provided by that type for the semantics defined by this class. I don't want to accidentally construct some other form of `Set`, for example, down in `putAllFirst`.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;
+    takeLock.lock();
+    try {
+      notEmpty.signal();
+    } finally {
+      takeLock.unlock();
+    }
+  }
+
+  /**
+   * Locks to prevent both puts and takes.
+   */
+  private void fullyLock() {
+    putLock.lock();
+    takeLock.lock();
+  }
+
+  /**
+   * Unlocks to allow both puts and takes.
+   */
+  private void fullyUnlock() {
+    takeLock.unlock();
+    putLock.unlock();
+  }
+
+  /**
+   * Inserts the specified element at the tail of the queue, if it's not already present.
+   *
+   * @param e the element to add
+   */
+  public void put(E e) {
+    if (e == null) {
+      throw new NullPointerException();
+    }
+
+    putLock.lock();
+    try {
+      delegate.add(e);
+    } finally {
+      putLock.unlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Inserts the specified element at the head of the queue.
+   *
+   * @param e the element to add
+   */
+  public void putFirst(E e) {
+    if (e == null) {
+      throw new NullPointerException();
+    }
+    putAllFirst(Collections.singleton(e));
+  }
+
+  /**
+   * Inserts the specified elements at the tail of the queue. Any elements already present in
+   * the queue are ignored.
+   *
+   * @param c the elements to add
+   */
+  public void putAll(Collection<? extends E> c) {
+    if (c == null) {
+      throw new NullPointerException();
+    }
+
+    putLock.lock();
+    try {
+      delegate.addAll(c);
+    } finally {
+      putLock.unlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Inserts the specified elements at the head of the queue.
+   *
+   * @param c the elements to add
+   */
+  public void putAllFirst(Collection<? extends E> c) {
+    if (c == null) {
+      throw new NullPointerException();
+    }
+
+    fullyLock();
+    try {
+      final LinkedHashSet<E> copy = new LinkedHashSet<>(c.size() + delegate.size());
+      copy.addAll(c);
+      copy.addAll(delegate);
+      delegate = copy;
+    } finally {
+      fullyUnlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Retrieves and removes the head of this queue, waiting if necessary
+   * until an element becomes available.
+   *
+   * @return the head of this queue
+   * @throws InterruptedException if interrupted while waiting
+   */
+  public E take() throws InterruptedException {
+    E x;
+    takeLock.lockInterruptibly();
+    try {
+      while (delegate.isEmpty()) {
+        notEmpty.await();
+      }
+      final Iterator<E> iter = delegate.iterator();
+      x = iter.next();
+      iter.remove();

Review comment:
       `putAllFirst` (and thus `putFirst`) and `clear` acquire both locks. `putAllFirst` is the only method that actually modifies the reference pointed to by `delegate`... maybe the `takeLock` isn't strictly needed for `clear`.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;

Review comment:
       This is a code-style that I copied from the reference implementation in `LinkedBlockingQueue`. I asked some jvm experts about this style and they told me it is done this way, particularly for locks, because the jvm doesn't enforce `final` on non-static fields. Which means that (1) `final` fields can be monkeyed with via reflection, but an inline variable declaration cannot be, and (2) a `final` field cannot be optimized by the compiler, while a `final` inline variable declaration can.
   
   TIL.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",

Review comment:
       The condition is examining the value in megabytes, while the warning is communicating to the operator in terms of the configuration point, which is in bytes. Thus, "MB" is the correct unit, and after division, `1mb` is the minimum effective allowable value.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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