mirror of
https://github.com/google/nomulus.git
synced 2025-07-23 19:20:44 +02:00
BSA for integration test (#2256)
Supports the full blocklist download cycle (download, diffing, diff-apply, and order-status reporting) and the refreshing of unblockable domains. Submitted due to tight deadline. We will conduct post-submit review and refactoring.
This commit is contained in:
parent
5315752bc0
commit
3f5c9d1246
68 changed files with 5191 additions and 260 deletions
|
@ -0,0 +1,43 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.util;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
import static com.google.common.collect.Iterators.partition;
|
||||
import static com.google.common.collect.Iterators.transform;
|
||||
import static com.google.common.collect.Streams.stream;
|
||||
import static java.lang.Math.min;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/** Utilities for breaking up a {@link Stream} into batches. */
|
||||
public final class BatchedStreams {
|
||||
|
||||
static final int MAX_BATCH = 1024 * 1024;
|
||||
|
||||
private BatchedStreams() {}
|
||||
|
||||
/**
|
||||
* Transform a flat {@link Stream} into a {@code Stream} of batches.
|
||||
*
|
||||
* <p>Closing the returned stream does not close the original stream.
|
||||
*/
|
||||
public static <T> Stream<ImmutableList<T>> toBatches(Stream<T> stream, int batchSize) {
|
||||
checkArgument(batchSize > 0, "batchSize must be a positive integer.");
|
||||
return stream(
|
||||
transform(partition(stream.iterator(), min(MAX_BATCH, batchSize)), ImmutableList::copyOf));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.util;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.util.BatchedStreams.toBatches;
|
||||
import static java.util.stream.Collectors.counting;
|
||||
import static java.util.stream.Collectors.groupingBy;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
/** Unit tests for {@link BatchedStreams}. */
|
||||
public class BatchedStreamsTest {
|
||||
|
||||
@Test
|
||||
void invalidBatchSize() {
|
||||
assertThat(assertThrows(IllegalArgumentException.class, () -> toBatches(Stream.of(), 0)))
|
||||
.hasMessageThat()
|
||||
.contains("must be a positive integer");
|
||||
}
|
||||
|
||||
@Test
|
||||
void batch_success() {
|
||||
// 900_002 elements -> 900 1K-batches + 1 2-element-batch
|
||||
Stream<Integer> data = IntStream.rangeClosed(0, 900_001).boxed();
|
||||
assertThat(
|
||||
toBatches(data, 1000).map(ImmutableList::size).collect(groupingBy(x -> x, counting())))
|
||||
.containsExactly(1000, 900L, 2, 1L);
|
||||
}
|
||||
|
||||
@Test
|
||||
void batch_partialBatch() {
|
||||
Stream<Integer> data = Stream.of(1, 2, 3);
|
||||
assertThat(
|
||||
toBatches(data, 1000).map(ImmutableList::size).collect(groupingBy(x -> x, counting())))
|
||||
.containsExactly(3, 1L);
|
||||
}
|
||||
|
||||
@Test
|
||||
void batch_truncateBatchSize() {
|
||||
// 2M elements -> 2 1M-batches despite the user-specified 2M batch size.
|
||||
Stream<Integer> data = IntStream.range(0, 1024 * 2048).boxed();
|
||||
assertThat(
|
||||
toBatches(data, 2_000_000)
|
||||
.map(ImmutableList::size)
|
||||
.collect(groupingBy(x -> x, counting())))
|
||||
.containsExactly(1024 * 1024, 2L);
|
||||
}
|
||||
}
|
164
core/src/main/java/google/registry/bsa/BlockListFetcher.java
Normal file
164
core/src/main/java/google/registry/bsa/BlockListFetcher.java
Normal file
|
@ -0,0 +1,164 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
|
||||
import com.google.api.client.http.HttpMethods;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import google.registry.bsa.api.BsaCredential;
|
||||
import google.registry.bsa.api.BsaException;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.request.UrlConnectionService;
|
||||
import google.registry.util.Retrier;
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URL;
|
||||
import java.security.GeneralSecurityException;
|
||||
import java.util.function.BiConsumer;
|
||||
import javax.inject.Inject;
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
|
||||
/** Fetches data from the BSA API. */
|
||||
public class BlockListFetcher {
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
private final UrlConnectionService urlConnectionService;
|
||||
private final BsaCredential credential;
|
||||
|
||||
private final ImmutableMap<String, String> blockListUrls;
|
||||
private final Retrier retrier;
|
||||
|
||||
@Inject
|
||||
BlockListFetcher(
|
||||
UrlConnectionService urlConnectionService,
|
||||
BsaCredential credential,
|
||||
@Config("bsaDataUrls") ImmutableMap<String, String> blockListUrls,
|
||||
Retrier retrier) {
|
||||
this.urlConnectionService = urlConnectionService;
|
||||
this.credential = credential;
|
||||
this.blockListUrls = blockListUrls;
|
||||
this.retrier = retrier;
|
||||
}
|
||||
|
||||
LazyBlockList fetch(BlockListType blockListType) {
|
||||
// TODO: use more informative exceptions to describe retriable errors
|
||||
return retrier.callWithRetry(
|
||||
() -> tryFetch(blockListType),
|
||||
e -> e instanceof BsaException && ((BsaException) e).isRetriable());
|
||||
}
|
||||
|
||||
LazyBlockList tryFetch(BlockListType blockListType) {
|
||||
try {
|
||||
URL dataUrl = new URL(blockListUrls.get(blockListType.name()));
|
||||
logger.atInfo().log("Downloading from %s", dataUrl);
|
||||
HttpsURLConnection connection =
|
||||
(HttpsURLConnection) urlConnectionService.createConnection(dataUrl);
|
||||
connection.setRequestMethod(HttpMethods.GET);
|
||||
connection.setRequestProperty("Authorization", "Bearer " + credential.getAuthToken());
|
||||
int code = connection.getResponseCode();
|
||||
if (code != SC_OK) {
|
||||
String errorDetails = "";
|
||||
try (InputStream errorStream = connection.getErrorStream()) {
|
||||
errorDetails = new String(ByteStreams.toByteArray(errorStream), UTF_8);
|
||||
} catch (NullPointerException e) {
|
||||
// No error message.
|
||||
} catch (Exception e) {
|
||||
errorDetails = "Failed to retrieve error message: " + e.getMessage();
|
||||
}
|
||||
throw new BsaException(
|
||||
String.format(
|
||||
"Status code: [%s], error: [%s], details: [%s]",
|
||||
code, connection.getResponseMessage(), errorDetails),
|
||||
/* retriable= */ true);
|
||||
}
|
||||
return new LazyBlockList(blockListType, connection);
|
||||
} catch (IOException e) {
|
||||
throw new BsaException(e, /* retriable= */ true);
|
||||
} catch (GeneralSecurityException e) {
|
||||
throw new BsaException(e, /* retriable= */ false);
|
||||
}
|
||||
}
|
||||
|
||||
static class LazyBlockList implements Closeable {
|
||||
|
||||
private final BlockListType blockListType;
|
||||
|
||||
private final HttpsURLConnection connection;
|
||||
|
||||
private final BufferedInputStream inputStream;
|
||||
private final String checksum;
|
||||
|
||||
LazyBlockList(BlockListType blockListType, HttpsURLConnection connection) throws IOException {
|
||||
this.blockListType = blockListType;
|
||||
this.connection = connection;
|
||||
this.inputStream = new BufferedInputStream(connection.getInputStream());
|
||||
this.checksum = readChecksum();
|
||||
}
|
||||
|
||||
/** Reads the BSA-generated checksum, which is the first line of the input. */
|
||||
private String readChecksum() throws IOException {
|
||||
StringBuilder checksum = new StringBuilder();
|
||||
char ch;
|
||||
while ((ch = peekInputStream()) != (char) -1 && !Character.isWhitespace(ch)) {
|
||||
checksum.append((char) inputStream.read());
|
||||
}
|
||||
while ((ch = peekInputStream()) != (char) -1 && Character.isWhitespace(ch)) {
|
||||
inputStream.read();
|
||||
}
|
||||
return checksum.toString();
|
||||
}
|
||||
|
||||
char peekInputStream() throws IOException {
|
||||
inputStream.mark(1);
|
||||
int byteValue = inputStream.read();
|
||||
inputStream.reset();
|
||||
return (char) byteValue;
|
||||
}
|
||||
|
||||
BlockListType getName() {
|
||||
return blockListType;
|
||||
}
|
||||
|
||||
String checksum() {
|
||||
return checksum;
|
||||
}
|
||||
|
||||
void consumeAll(BiConsumer<byte[], Integer> consumer) throws IOException {
|
||||
byte[] buffer = new byte[1024];
|
||||
int bytesRead;
|
||||
while ((bytesRead = inputStream.read(buffer)) != -1) {
|
||||
consumer.accept(buffer, bytesRead);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (inputStream != null) {
|
||||
try {
|
||||
inputStream.close();
|
||||
} catch (IOException e) {
|
||||
// Fall through to close the connection.
|
||||
}
|
||||
}
|
||||
connection.disconnect();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -14,8 +14,10 @@
|
|||
|
||||
package google.registry.bsa;
|
||||
|
||||
/** Identifiers of the BSA lists with blocking labels. */
|
||||
public enum BlockList {
|
||||
/**
|
||||
* The product types of the block lists, which determines the http endpoint that serves the data.
|
||||
*/
|
||||
public enum BlockListType {
|
||||
BLOCK,
|
||||
BLOCK_PLUS;
|
||||
}
|
267
core/src/main/java/google/registry/bsa/BsaDiffCreator.java
Normal file
267
core/src/main/java/google/registry/bsa/BsaDiffCreator.java
Normal file
|
@ -0,0 +1,267 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||
import static com.google.common.collect.Maps.newHashMap;
|
||||
import static com.google.common.collect.Multimaps.newListMultimap;
|
||||
import static com.google.common.collect.Multimaps.toMultimap;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMultimap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.google.common.collect.Sets;
|
||||
import google.registry.bsa.api.BlockLabel;
|
||||
import google.registry.bsa.api.BlockLabel.LabelType;
|
||||
import google.registry.bsa.api.BlockOrder;
|
||||
import google.registry.bsa.api.BlockOrder.OrderType;
|
||||
import google.registry.bsa.persistence.DownloadSchedule;
|
||||
import google.registry.bsa.persistence.DownloadSchedule.CompletedJob;
|
||||
import google.registry.tldconfig.idn.IdnTableEnum;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
import javax.inject.Inject;
|
||||
|
||||
/** Creates diffs between the most recent download and the previous one. */
|
||||
class BsaDiffCreator {
|
||||
|
||||
private static final Splitter LINE_SPLITTER = Splitter.on(',').trimResults();
|
||||
private static final Splitter ORDER_SPLITTER = Splitter.on(';').trimResults();
|
||||
|
||||
private static final String BSA_CSV_HEADER = "domainLabel,orderIDs";
|
||||
|
||||
/** An impossible value for order ID. See {@link #createDiff} for usage. */
|
||||
static final Long ORDER_ID_SENTINEL = Long.MIN_VALUE;
|
||||
|
||||
private final GcsClient gcsClient;
|
||||
|
||||
@Inject
|
||||
BsaDiffCreator(GcsClient gcsClient) {
|
||||
this.gcsClient = gcsClient;
|
||||
}
|
||||
|
||||
private <K, V extends Comparable> Multimap<K, V> listBackedMultiMap() {
|
||||
return newListMultimap(newHashMap(), Lists::newArrayList);
|
||||
}
|
||||
|
||||
BsaDiff createDiff(DownloadSchedule schedule, IdnChecker idnChecker) {
|
||||
String currentJobName = schedule.jobName();
|
||||
Optional<String> previousJobName = schedule.latestCompleted().map(CompletedJob::jobName);
|
||||
/**
|
||||
* Memory usage is a concern when creating a diff, when the newest download needs to be held in
|
||||
* memory in its entirety. The top-grade AppEngine VM has 3GB of memory, leaving less than 1.5GB
|
||||
* to application memory footprint after subtracting overheads due to copying garbage collection
|
||||
* and non-heap data etc. Assuming 400K labels, each of which on average included in 5 orders,
|
||||
* the memory footprint is at least 300MB when loaded into a Hashset-backed Multimap (64-bit
|
||||
* JVM, with 12-byte object header, 16-byte array header, and 16-byte alignment).
|
||||
*
|
||||
* <p>The memory footprint can be reduced in two ways, by using a canonical instance for each
|
||||
* order ID value, and by using a ArrayList-backed Multimap. Together they reduce memory size to
|
||||
* well below 100MB for the scenario above.
|
||||
*
|
||||
* <p>We need to watch out for the download sizes even after the migration to GKE. However, at
|
||||
* that point we will have a wider selection of hardware.
|
||||
*
|
||||
* <p>Beam pipeline is not a good option. It has to be launched as a separate, asynchronous job,
|
||||
* and there is no guaranteed limit to launch delay. Both issues would increase code complexity.
|
||||
*/
|
||||
Canonicals<Long> canonicals = new Canonicals<>();
|
||||
try (Stream<Line> currentStream = loadBlockLists(currentJobName);
|
||||
Stream<Line> previousStream =
|
||||
previousJobName.map(this::loadBlockLists).orElseGet(Stream::of)) {
|
||||
/**
|
||||
* Load current label/order pairs into a multimap, which will contain both new labels and
|
||||
* those that stay on when processing is done.
|
||||
*/
|
||||
Multimap<String, Long> newAndRemaining =
|
||||
currentStream
|
||||
.map(line -> line.labelOrderPairs(canonicals))
|
||||
.flatMap(x -> x)
|
||||
.collect(
|
||||
toMultimap(
|
||||
LabelOrderPair::label, LabelOrderPair::orderId, this::listBackedMultiMap));
|
||||
|
||||
Multimap<String, Long> deleted =
|
||||
previousStream
|
||||
.map(
|
||||
line -> {
|
||||
// Mark labels that exist in both downloads with the SENTINEL id. This helps
|
||||
// distinguish existing label with new order from new labels.
|
||||
if (newAndRemaining.containsKey(line.label())
|
||||
&& !newAndRemaining.containsEntry(line.label(), ORDER_ID_SENTINEL)) {
|
||||
newAndRemaining.put(line.label(), ORDER_ID_SENTINEL);
|
||||
}
|
||||
return line;
|
||||
})
|
||||
.map(line -> line.labelOrderPairs(canonicals))
|
||||
.flatMap(x -> x)
|
||||
.filter(kv -> !newAndRemaining.remove(kv.label(), kv.orderId()))
|
||||
.collect(
|
||||
toMultimap(
|
||||
LabelOrderPair::label, LabelOrderPair::orderId, this::listBackedMultiMap));
|
||||
|
||||
/**
|
||||
* Labels in `newAndRemaining`:
|
||||
*
|
||||
* <ul>
|
||||
* <li>Mapped to `sentinel` only: Labels without change, ignore
|
||||
* <li>Mapped to `sentinel` and some orders: Existing labels with new order mapping. Those
|
||||
* orders are new orders.
|
||||
* <li>Mapped to some orders but not `sentinel`: New labels and new orders.
|
||||
* </ul>
|
||||
*
|
||||
* <p>The `deleted` map has
|
||||
*
|
||||
* <ul>
|
||||
* <li>Deleted labels: the keyset of deleted minus the keyset of the newAndRemaining
|
||||
* <li>Deleted orders: the union of values.
|
||||
* </ul>
|
||||
*/
|
||||
return new BsaDiff(
|
||||
ImmutableMultimap.copyOf(newAndRemaining), ImmutableMultimap.copyOf(deleted), idnChecker);
|
||||
}
|
||||
}
|
||||
|
||||
Stream<Line> loadBlockLists(String jobName) {
|
||||
return Stream.of(BlockListType.values())
|
||||
.map(blockList -> gcsClient.readBlockList(jobName, blockList))
|
||||
.flatMap(x -> x)
|
||||
.filter(line -> !line.startsWith(BSA_CSV_HEADER))
|
||||
.map(BsaDiffCreator::parseLine);
|
||||
}
|
||||
|
||||
static Line parseLine(String line) {
|
||||
List<String> columns = LINE_SPLITTER.splitToList(line);
|
||||
checkArgument(columns.size() == 2, "Invalid line: [%s]", line);
|
||||
checkArgument(!Strings.isNullOrEmpty(columns.get(0)), "Missing label in line: [%s]", line);
|
||||
try {
|
||||
ImmutableList<Long> orderIds =
|
||||
ORDER_SPLITTER
|
||||
.splitToStream(columns.get(1))
|
||||
.map(Long::valueOf)
|
||||
.collect(toImmutableList());
|
||||
checkArgument(!orderIds.isEmpty(), "Missing orders in line: [%s]", line);
|
||||
checkArgument(
|
||||
!orderIds.contains(ORDER_ID_SENTINEL), "Invalid order id %s", ORDER_ID_SENTINEL);
|
||||
return Line.of(columns.get(0), orderIds);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException(line, e);
|
||||
}
|
||||
}
|
||||
|
||||
static class BsaDiff {
|
||||
private final ImmutableMultimap<String, Long> newAndRemaining;
|
||||
|
||||
private final ImmutableMultimap<String, Long> deleted;
|
||||
private final IdnChecker idnChecker;
|
||||
|
||||
BsaDiff(
|
||||
ImmutableMultimap<String, Long> newAndRemaining,
|
||||
ImmutableMultimap<String, Long> deleted,
|
||||
IdnChecker idnChecker) {
|
||||
this.newAndRemaining = newAndRemaining;
|
||||
this.deleted = deleted;
|
||||
this.idnChecker = idnChecker;
|
||||
}
|
||||
|
||||
Stream<BlockOrder> getOrders() {
|
||||
return Stream.concat(
|
||||
newAndRemaining.values().stream()
|
||||
.filter(value -> !Objects.equals(ORDER_ID_SENTINEL, value))
|
||||
.distinct()
|
||||
.map(id -> BlockOrder.of(id, OrderType.CREATE)),
|
||||
deleted.values().stream().distinct().map(id -> BlockOrder.of(id, OrderType.DELETE)));
|
||||
}
|
||||
|
||||
Stream<BlockLabel> getLabels() {
|
||||
return Stream.of(
|
||||
newAndRemaining.asMap().entrySet().stream()
|
||||
.filter(e -> e.getValue().size() > 1 || !e.getValue().contains(ORDER_ID_SENTINEL))
|
||||
.filter(entry -> entry.getValue().contains(ORDER_ID_SENTINEL))
|
||||
.map(
|
||||
entry ->
|
||||
BlockLabel.of(
|
||||
entry.getKey(),
|
||||
LabelType.NEW_ORDER_ASSOCIATION,
|
||||
idnChecker.getAllValidIdns(entry.getKey()).stream()
|
||||
.map(IdnTableEnum::name)
|
||||
.collect(toImmutableSet()))),
|
||||
newAndRemaining.asMap().entrySet().stream()
|
||||
.filter(e -> e.getValue().size() > 1 || !e.getValue().contains(ORDER_ID_SENTINEL))
|
||||
.filter(entry -> !entry.getValue().contains(ORDER_ID_SENTINEL))
|
||||
.map(
|
||||
entry ->
|
||||
BlockLabel.of(
|
||||
entry.getKey(),
|
||||
LabelType.CREATE,
|
||||
idnChecker.getAllValidIdns(entry.getKey()).stream()
|
||||
.map(IdnTableEnum::name)
|
||||
.collect(toImmutableSet()))),
|
||||
Sets.difference(deleted.keySet(), newAndRemaining.keySet()).stream()
|
||||
.map(label -> BlockLabel.of(label, LabelType.DELETE, ImmutableSet.of())))
|
||||
.flatMap(x -> x);
|
||||
}
|
||||
}
|
||||
|
||||
static class Canonicals<T> {
|
||||
private final HashMap<T, T> cache;
|
||||
|
||||
Canonicals() {
|
||||
cache = Maps.newHashMap();
|
||||
}
|
||||
|
||||
T get(T value) {
|
||||
cache.putIfAbsent(value, value);
|
||||
return cache.get(value);
|
||||
}
|
||||
}
|
||||
|
||||
@AutoValue
|
||||
abstract static class LabelOrderPair {
|
||||
abstract String label();
|
||||
|
||||
abstract Long orderId();
|
||||
|
||||
static <K, V> LabelOrderPair of(String key, Long value) {
|
||||
return new AutoValue_BsaDiffCreator_LabelOrderPair(key, value);
|
||||
}
|
||||
}
|
||||
|
||||
@AutoValue
|
||||
abstract static class Line {
|
||||
abstract String label();
|
||||
|
||||
abstract ImmutableList<Long> orderIds();
|
||||
|
||||
Stream<LabelOrderPair> labelOrderPairs(Canonicals<Long> canonicals) {
|
||||
return orderIds().stream().map(id -> LabelOrderPair.of(label(), canonicals.get(id)));
|
||||
}
|
||||
|
||||
static Line of(String label, ImmutableList<Long> orderIds) {
|
||||
return new AutoValue_BsaDiffCreator_Line(label, orderIds);
|
||||
}
|
||||
}
|
||||
}
|
237
core/src/main/java/google/registry/bsa/BsaDownloadAction.java
Normal file
237
core/src/main/java/google/registry/bsa/BsaDownloadAction.java
Normal file
|
@ -0,0 +1,237 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static google.registry.bsa.BlockListType.BLOCK;
|
||||
import static google.registry.bsa.BlockListType.BLOCK_PLUS;
|
||||
import static google.registry.bsa.api.JsonSerializations.toCompletedOrdersReport;
|
||||
import static google.registry.bsa.api.JsonSerializations.toInProgressOrdersReport;
|
||||
import static google.registry.bsa.api.JsonSerializations.toUnblockableDomainsReport;
|
||||
import static google.registry.bsa.persistence.LabelDiffUpdates.applyLabelDiff;
|
||||
import static google.registry.request.Action.Method.GET;
|
||||
import static google.registry.request.Action.Method.POST;
|
||||
import static google.registry.util.BatchedStreams.toBatches;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import dagger.Lazy;
|
||||
import google.registry.bsa.BlockListFetcher.LazyBlockList;
|
||||
import google.registry.bsa.BsaDiffCreator.BsaDiff;
|
||||
import google.registry.bsa.api.BlockLabel;
|
||||
import google.registry.bsa.api.BlockOrder;
|
||||
import google.registry.bsa.api.BsaReportSender;
|
||||
import google.registry.bsa.api.UnblockableDomain;
|
||||
import google.registry.bsa.persistence.DownloadSchedule;
|
||||
import google.registry.bsa.persistence.DownloadScheduler;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.model.tld.Tlds;
|
||||
import google.registry.request.Action;
|
||||
import google.registry.request.Response;
|
||||
import google.registry.request.auth.Auth;
|
||||
import google.registry.util.Clock;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
import javax.inject.Inject;
|
||||
|
||||
@Action(
|
||||
service = Action.Service.BSA,
|
||||
path = BsaDownloadAction.PATH,
|
||||
method = {GET, POST},
|
||||
auth = Auth.AUTH_API_ADMIN)
|
||||
public class BsaDownloadAction implements Runnable {
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
static final String PATH = "/_dr/task/bsaDownload";
|
||||
|
||||
private final DownloadScheduler downloadScheduler;
|
||||
private final BlockListFetcher blockListFetcher;
|
||||
private final BsaDiffCreator diffCreator;
|
||||
private final BsaReportSender bsaReportSender;
|
||||
private final GcsClient gcsClient;
|
||||
private final Lazy<IdnChecker> lazyIdnChecker;
|
||||
private final BsaLock bsaLock;
|
||||
private final Clock clock;
|
||||
private final int transactionBatchSize;
|
||||
private final Response response;
|
||||
|
||||
@Inject
|
||||
BsaDownloadAction(
|
||||
DownloadScheduler downloadScheduler,
|
||||
BlockListFetcher blockListFetcher,
|
||||
BsaDiffCreator diffCreator,
|
||||
BsaReportSender bsaReportSender,
|
||||
GcsClient gcsClient,
|
||||
Lazy<IdnChecker> lazyIdnChecker,
|
||||
BsaLock bsaLock,
|
||||
Clock clock,
|
||||
@Config("bsaTxnBatchSize") int transactionBatchSize,
|
||||
Response response) {
|
||||
this.downloadScheduler = downloadScheduler;
|
||||
this.blockListFetcher = blockListFetcher;
|
||||
this.diffCreator = diffCreator;
|
||||
this.bsaReportSender = bsaReportSender;
|
||||
this.gcsClient = gcsClient;
|
||||
this.lazyIdnChecker = lazyIdnChecker;
|
||||
this.bsaLock = bsaLock;
|
||||
this.clock = clock;
|
||||
this.transactionBatchSize = transactionBatchSize;
|
||||
this.response = response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
if (!bsaLock.executeWithLock(this::runWithinLock)) {
|
||||
logger.atInfo().log("Job is being executed by another worker.");
|
||||
}
|
||||
} catch (Throwable throwable) {
|
||||
// TODO(12/31/2023): consider sending an alert email.
|
||||
// TODO: if unretriable errors, log at severe and send email.
|
||||
logger.atWarning().withCause(throwable).log("Failed to update block lists.");
|
||||
}
|
||||
// Always return OK. Let the next cron job retry.
|
||||
response.setStatus(SC_OK);
|
||||
}
|
||||
|
||||
Void runWithinLock() {
|
||||
// Cannot enroll new TLDs after download starts. This may change if b/309175410 is fixed.
|
||||
if (!Tlds.hasActiveBsaEnrollment(clock.nowUtc())) {
|
||||
logger.atInfo().log("No TLDs enrolled with BSA. Quitting.");
|
||||
return null;
|
||||
}
|
||||
Optional<DownloadSchedule> scheduleOptional = downloadScheduler.schedule();
|
||||
if (!scheduleOptional.isPresent()) {
|
||||
logger.atInfo().log("Nothing to do.");
|
||||
return null;
|
||||
}
|
||||
BsaDiff diff = null;
|
||||
DownloadSchedule schedule = scheduleOptional.get();
|
||||
switch (schedule.stage()) {
|
||||
case DOWNLOAD_BLOCK_LISTS:
|
||||
try (LazyBlockList block = blockListFetcher.fetch(BLOCK);
|
||||
LazyBlockList blockPlus = blockListFetcher.fetch(BLOCK_PLUS)) {
|
||||
ImmutableMap<BlockListType, String> fetchedChecksums =
|
||||
ImmutableMap.of(BLOCK, block.checksum(), BLOCK_PLUS, blockPlus.checksum());
|
||||
ImmutableMap<BlockListType, String> prevChecksums =
|
||||
schedule
|
||||
.latestCompleted()
|
||||
.map(DownloadSchedule.CompletedJob::checksums)
|
||||
.orElseGet(ImmutableMap::of);
|
||||
boolean checksumsMatch = Objects.equals(fetchedChecksums, prevChecksums);
|
||||
if (!schedule.alwaysDownload() && checksumsMatch) {
|
||||
logger.atInfo().log(
|
||||
"Skipping download b/c block list checksums have not changed: [%s]",
|
||||
fetchedChecksums);
|
||||
schedule.updateJobStage(DownloadStage.NOP, fetchedChecksums);
|
||||
return null;
|
||||
} else if (checksumsMatch) {
|
||||
logger.atInfo().log(
|
||||
"Checksums match but download anyway: elapsed time since last download exceeds"
|
||||
+ " configured limit.");
|
||||
}
|
||||
// When downloading, always fetch both lists so that whole data set is in one GCS folder.
|
||||
ImmutableMap<BlockListType, String> actualChecksum =
|
||||
gcsClient.saveAndChecksumBlockList(
|
||||
schedule.jobName(), ImmutableList.of(block, blockPlus));
|
||||
if (!Objects.equals(fetchedChecksums, actualChecksum)) {
|
||||
logger.atSevere().log(
|
||||
"Inlined checksums do not match those calculated by us. Theirs: [%s]; ours: [%s]",
|
||||
fetchedChecksums, actualChecksum);
|
||||
schedule.updateJobStage(DownloadStage.CHECKSUMS_DO_NOT_MATCH, fetchedChecksums);
|
||||
// TODO(01/15/24): add email alert.
|
||||
return null;
|
||||
}
|
||||
schedule.updateJobStage(DownloadStage.MAKE_ORDER_AND_LABEL_DIFF, actualChecksum);
|
||||
}
|
||||
// Fall through
|
||||
case MAKE_ORDER_AND_LABEL_DIFF:
|
||||
diff = diffCreator.createDiff(schedule, lazyIdnChecker.get());
|
||||
gcsClient.writeOrderDiffs(schedule.jobName(), diff.getOrders());
|
||||
gcsClient.writeLabelDiffs(schedule.jobName(), diff.getLabels());
|
||||
schedule.updateJobStage(DownloadStage.APPLY_ORDER_AND_LABEL_DIFF);
|
||||
// Fall through
|
||||
case APPLY_ORDER_AND_LABEL_DIFF:
|
||||
try (Stream<BlockLabel> labels =
|
||||
diff != null ? diff.getLabels() : gcsClient.readLabelDiffs(schedule.jobName())) {
|
||||
Stream<ImmutableList<BlockLabel>> batches = toBatches(labels, transactionBatchSize);
|
||||
gcsClient.writeUnblockableDomains(
|
||||
schedule.jobName(),
|
||||
batches
|
||||
.map(
|
||||
batch ->
|
||||
applyLabelDiff(batch, lazyIdnChecker.get(), schedule, clock.nowUtc()))
|
||||
.flatMap(ImmutableList::stream));
|
||||
}
|
||||
schedule.updateJobStage(DownloadStage.REPORT_START_OF_ORDER_PROCESSING);
|
||||
// Fall through
|
||||
case REPORT_START_OF_ORDER_PROCESSING:
|
||||
try (Stream<BlockOrder> orders = gcsClient.readOrderDiffs(schedule.jobName())) {
|
||||
// We expect that all order instances and the json string can fit in memory.
|
||||
Optional<String> report = toInProgressOrdersReport(orders);
|
||||
if (report.isPresent()) {
|
||||
// Log report data
|
||||
gcsClient.logInProgressOrderReport(
|
||||
schedule.jobName(), BsaStringUtils.LINE_SPLITTER.splitToStream(report.get()));
|
||||
bsaReportSender.sendOrderStatusReport(report.get());
|
||||
} else {
|
||||
logger.atInfo().log("No new or deleted orders in this round.");
|
||||
}
|
||||
}
|
||||
schedule.updateJobStage(DownloadStage.UPLOAD_UNBLOCKABLE_DOMAINS_FOR_NEW_ORDERS);
|
||||
// Fall through
|
||||
case UPLOAD_UNBLOCKABLE_DOMAINS_FOR_NEW_ORDERS:
|
||||
try (Stream<UnblockableDomain> unblockables =
|
||||
gcsClient.readUnblockableDomains(schedule.jobName())) {
|
||||
/* The number of unblockable domains may be huge in theory (label x ~50 tlds), but in
|
||||
* practice should be relatively small (tens of thousands?). Batches can be introduced
|
||||
* if size becomes a problem.
|
||||
*/
|
||||
Optional<String> report = toUnblockableDomainsReport(unblockables);
|
||||
if (report.isPresent()) {
|
||||
gcsClient.logAddedUnblockableDomainsReport(
|
||||
schedule.jobName(), BsaStringUtils.LINE_SPLITTER.splitToStream(report.get()));
|
||||
// During downloads, unblockable domains are only added, not removed.
|
||||
bsaReportSender.addUnblockableDomainsUpdates(report.get());
|
||||
} else {
|
||||
logger.atInfo().log("No changes in the set of unblockable domains in this round.");
|
||||
}
|
||||
}
|
||||
schedule.updateJobStage(DownloadStage.REPORT_END_OF_ORDER_PROCESSING);
|
||||
// Fall through
|
||||
case REPORT_END_OF_ORDER_PROCESSING:
|
||||
try (Stream<BlockOrder> orders = gcsClient.readOrderDiffs(schedule.jobName())) {
|
||||
// Orders are expected to be few, so the report can be kept in memory.
|
||||
Optional<String> report = toCompletedOrdersReport(orders);
|
||||
if (report.isPresent()) {
|
||||
gcsClient.logCompletedOrderReport(
|
||||
schedule.jobName(), BsaStringUtils.LINE_SPLITTER.splitToStream(report.get()));
|
||||
bsaReportSender.sendOrderStatusReport(report.get());
|
||||
}
|
||||
}
|
||||
schedule.updateJobStage(DownloadStage.DONE);
|
||||
return null;
|
||||
case DONE:
|
||||
case NOP:
|
||||
case CHECKSUMS_DO_NOT_MATCH:
|
||||
logger.atWarning().log("Unexpectedly reached the %s stage.", schedule.stage());
|
||||
break;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -14,32 +14,27 @@
|
|||
|
||||
package google.registry.bsa;
|
||||
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
|
||||
import google.registry.request.Action;
|
||||
import google.registry.request.Action.Service;
|
||||
import google.registry.request.Response;
|
||||
import google.registry.request.auth.Auth;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.request.lock.LockHandler;
|
||||
import java.util.concurrent.Callable;
|
||||
import javax.inject.Inject;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
@Action(
|
||||
service = Service.BSA,
|
||||
path = PlaceholderAction.PATH,
|
||||
method = Action.Method.GET,
|
||||
auth = Auth.AUTH_API_ADMIN)
|
||||
public class PlaceholderAction implements Runnable {
|
||||
private final Response response;
|
||||
/** Helper for guarding all BSA related work with a common lock. */
|
||||
public class BsaLock {
|
||||
|
||||
static final String PATH = "/_dr/task/bsaDownload";
|
||||
private static final String LOCK_NAME = "all-bsa-jobs";
|
||||
|
||||
private final LockHandler lockHandler;
|
||||
private final Duration leaseExpiry;
|
||||
|
||||
@Inject
|
||||
public PlaceholderAction(Response response) {
|
||||
this.response = response;
|
||||
BsaLock(LockHandler lockHandler, @Config("bsaLockLeaseExpiry") Duration leaseExpiry) {
|
||||
this.lockHandler = lockHandler;
|
||||
this.leaseExpiry = leaseExpiry;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
response.setStatus(SC_OK);
|
||||
response.setPayload("Hello World");
|
||||
boolean executeWithLock(Callable<Void> callable) {
|
||||
return lockHandler.executeWithLocks(callable, null, leaseExpiry, LOCK_NAME);
|
||||
}
|
||||
}
|
174
core/src/main/java/google/registry/bsa/BsaRefreshAction.java
Normal file
174
core/src/main/java/google/registry/bsa/BsaRefreshAction.java
Normal file
|
@ -0,0 +1,174 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static google.registry.bsa.BsaStringUtils.LINE_SPLITTER;
|
||||
import static google.registry.request.Action.Method.GET;
|
||||
import static google.registry.request.Action.Method.POST;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import google.registry.bsa.api.BsaReportSender;
|
||||
import google.registry.bsa.api.JsonSerializations;
|
||||
import google.registry.bsa.api.UnblockableDomainChange;
|
||||
import google.registry.bsa.persistence.DomainsRefresher;
|
||||
import google.registry.bsa.persistence.RefreshSchedule;
|
||||
import google.registry.bsa.persistence.RefreshScheduler;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.model.tld.Tlds;
|
||||
import google.registry.request.Action;
|
||||
import google.registry.request.Response;
|
||||
import google.registry.request.auth.Auth;
|
||||
import google.registry.util.BatchedStreams;
|
||||
import google.registry.util.Clock;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
import javax.inject.Inject;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
@Action(
|
||||
service = Action.Service.BSA,
|
||||
path = BsaRefreshAction.PATH,
|
||||
method = {GET, POST},
|
||||
auth = Auth.AUTH_API_ADMIN)
|
||||
public class BsaRefreshAction implements Runnable {
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
static final String PATH = "/_dr/task/bsaRefresh";
|
||||
|
||||
private final RefreshScheduler scheduler;
|
||||
private final GcsClient gcsClient;
|
||||
private final BsaReportSender bsaReportSender;
|
||||
private final int transactionBatchSize;
|
||||
private final Duration domainTxnMaxDuration;
|
||||
private final BsaLock bsaLock;
|
||||
private final Clock clock;
|
||||
private final Response response;
|
||||
|
||||
@Inject
|
||||
BsaRefreshAction(
|
||||
RefreshScheduler scheduler,
|
||||
GcsClient gcsClient,
|
||||
BsaReportSender bsaReportSender,
|
||||
@Config("bsaTxnBatchSize") int transactionBatchSize,
|
||||
@Config("domainTxnMaxDuration") Duration domainTxnMaxDuration,
|
||||
BsaLock bsaLock,
|
||||
Clock clock,
|
||||
Response response) {
|
||||
this.scheduler = scheduler;
|
||||
this.gcsClient = gcsClient;
|
||||
this.bsaReportSender = bsaReportSender;
|
||||
this.transactionBatchSize = transactionBatchSize;
|
||||
this.domainTxnMaxDuration = domainTxnMaxDuration;
|
||||
this.bsaLock = bsaLock;
|
||||
this.clock = clock;
|
||||
this.response = response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
if (!bsaLock.executeWithLock(this::runWithinLock)) {
|
||||
logger.atInfo().log("Job is being executed by another worker.");
|
||||
}
|
||||
} catch (Throwable throwable) {
|
||||
// TODO(12/31/2023): consider sending an alert email.
|
||||
logger.atWarning().withCause(throwable).log("Failed to update block lists.");
|
||||
}
|
||||
// Always return OK. No need to use a retrier on `runWithinLock`. Its individual steps are
|
||||
// implicitly retried. If action fails, the next cron will continue at checkpoint.
|
||||
response.setStatus(SC_OK);
|
||||
}
|
||||
|
||||
/** Executes the refresh action while holding the BSA lock. */
|
||||
Void runWithinLock() {
|
||||
// Cannot enroll new TLDs after download starts. This may change if b/309175410 is fixed.
|
||||
if (!Tlds.hasActiveBsaEnrollment(clock.nowUtc())) {
|
||||
logger.atInfo().log("No TLDs enrolled with BSA. Quitting.");
|
||||
return null;
|
||||
}
|
||||
Optional<RefreshSchedule> maybeSchedule = scheduler.schedule();
|
||||
if (!maybeSchedule.isPresent()) {
|
||||
logger.atInfo().log("No completed downloads yet. Exiting.");
|
||||
return null;
|
||||
}
|
||||
RefreshSchedule schedule = maybeSchedule.get();
|
||||
DomainsRefresher refresher =
|
||||
new DomainsRefresher(
|
||||
schedule.prevRefreshTime(), clock.nowUtc(), domainTxnMaxDuration, transactionBatchSize);
|
||||
switch (schedule.stage()) {
|
||||
case CHECK_FOR_CHANGES:
|
||||
ImmutableList<UnblockableDomainChange> blockabilityChanges =
|
||||
refresher.checkForBlockabilityChanges();
|
||||
if (blockabilityChanges.isEmpty()) {
|
||||
logger.atInfo().log("No change to Unblockable domains found.");
|
||||
schedule.updateJobStage(RefreshStage.DONE);
|
||||
return null;
|
||||
}
|
||||
gcsClient.writeRefreshChanges(schedule.jobName(), blockabilityChanges.stream());
|
||||
schedule.updateJobStage(RefreshStage.APPLY_CHANGES);
|
||||
// Fall through
|
||||
case APPLY_CHANGES:
|
||||
try (Stream<UnblockableDomainChange> changes =
|
||||
gcsClient.readRefreshChanges(schedule.jobName())) {
|
||||
BatchedStreams.toBatches(changes, 500).forEach(refresher::applyUnblockableChanges);
|
||||
}
|
||||
schedule.updateJobStage(RefreshStage.UPLOAD_REMOVALS);
|
||||
// Fall through
|
||||
case UPLOAD_REMOVALS:
|
||||
try (Stream<UnblockableDomainChange> changes =
|
||||
gcsClient.readRefreshChanges(schedule.jobName())) {
|
||||
Optional<String> report =
|
||||
JsonSerializations.toUnblockableDomainsRemovalReport(
|
||||
changes
|
||||
.filter(UnblockableDomainChange::isDelete)
|
||||
.map(UnblockableDomainChange::domainName));
|
||||
if (report.isPresent()) {
|
||||
gcsClient.logRemovedUnblockableDomainsReport(
|
||||
schedule.jobName(), LINE_SPLITTER.splitToStream(report.get()));
|
||||
bsaReportSender.removeUnblockableDomainsUpdates(report.get());
|
||||
} else {
|
||||
logger.atInfo().log("No Unblockable domains to remove.");
|
||||
}
|
||||
}
|
||||
schedule.updateJobStage(RefreshStage.UPLOAD_ADDITIONS);
|
||||
// Fall through
|
||||
case UPLOAD_ADDITIONS:
|
||||
try (Stream<UnblockableDomainChange> changes =
|
||||
gcsClient.readRefreshChanges(schedule.jobName())) {
|
||||
Optional<String> report =
|
||||
JsonSerializations.toUnblockableDomainsReport(
|
||||
changes
|
||||
.filter(UnblockableDomainChange::AddOrChange)
|
||||
.map(UnblockableDomainChange::newValue));
|
||||
if (report.isPresent()) {
|
||||
gcsClient.logRemovedUnblockableDomainsReport(
|
||||
schedule.jobName(), LINE_SPLITTER.splitToStream(report.get()));
|
||||
bsaReportSender.removeUnblockableDomainsUpdates(report.get());
|
||||
} else {
|
||||
logger.atInfo().log("No new Unblockable domains to add.");
|
||||
}
|
||||
}
|
||||
schedule.updateJobStage(RefreshStage.DONE);
|
||||
break;
|
||||
case DONE:
|
||||
logger.atInfo().log("Unexpectedly reaching the `DONE` stage.");
|
||||
break;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
45
core/src/main/java/google/registry/bsa/BsaStringUtils.java
Normal file
45
core/src/main/java/google/registry/bsa/BsaStringUtils.java
Normal file
|
@ -0,0 +1,45 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Splitter;
|
||||
import java.util.List;
|
||||
|
||||
/** Helpers for domain name manipulation and string serialization of Java objects. */
|
||||
public class BsaStringUtils {
|
||||
|
||||
public static final Joiner DOMAIN_JOINER = Joiner.on('.');
|
||||
public static final Joiner PROPERTY_JOINER = Joiner.on(',');
|
||||
public static final Splitter DOMAIN_SPLITTER = Splitter.on('.');
|
||||
public static final Splitter PROPERTY_SPLITTER = Splitter.on(',');
|
||||
public static final Splitter LINE_SPLITTER = Splitter.on('\n');
|
||||
|
||||
public static String getLabelInDomain(String domainName) {
|
||||
List<String> parts = DOMAIN_SPLITTER.limit(1).splitToList(domainName);
|
||||
checkArgument(!parts.isEmpty(), "Not a valid domain: [%s]", domainName);
|
||||
return parts.get(0);
|
||||
}
|
||||
|
||||
public static String getTldInDomain(String domainName) {
|
||||
List<String> parts = DOMAIN_SPLITTER.splitToList(domainName);
|
||||
checkArgument(parts.size() == 2, "Not a valid domain: [%s]", domainName);
|
||||
return parts.get(1);
|
||||
}
|
||||
|
||||
private BsaStringUtils() {}
|
||||
}
|
42
core/src/main/java/google/registry/bsa/BsaTransactions.java
Normal file
42
core/src/main/java/google/registry/bsa/BsaTransactions.java
Normal file
|
@ -0,0 +1,42 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static google.registry.persistence.PersistenceModule.TransactionIsolationLevel.TRANSACTION_REPEATABLE_READ;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
|
||||
import com.google.errorprone.annotations.CanIgnoreReturnValue;
|
||||
import java.util.concurrent.Callable;
|
||||
|
||||
/**
|
||||
* Helpers for executing JPA transactions for BSA processing.
|
||||
*
|
||||
* <p>All mutating transactions for BSA may be executed at the {@code TRANSACTION_REPEATABLE_READ}
|
||||
* level.
|
||||
*/
|
||||
public final class BsaTransactions {
|
||||
|
||||
@CanIgnoreReturnValue
|
||||
public static <T> T bsaTransact(Callable<T> work) {
|
||||
return tm().transact(work, TRANSACTION_REPEATABLE_READ);
|
||||
}
|
||||
|
||||
@CanIgnoreReturnValue
|
||||
public static <T> T bsaQuery(Callable<T> work) {
|
||||
return tm().transact(work, TRANSACTION_REPEATABLE_READ);
|
||||
}
|
||||
|
||||
private BsaTransactions() {}
|
||||
}
|
|
@ -14,23 +14,32 @@
|
|||
|
||||
package google.registry.bsa;
|
||||
|
||||
import google.registry.bsa.api.BlockLabel;
|
||||
import google.registry.bsa.api.BlockOrder;
|
||||
|
||||
/** The processing stages of a download. */
|
||||
public enum DownloadStage {
|
||||
/** Downloads BSA block list files. */
|
||||
DOWNLOAD,
|
||||
/** Generates block list diffs with the previous download. */
|
||||
MAKE_DIFF,
|
||||
/** Applies the label diffs to the database tables. */
|
||||
APPLY_DIFF,
|
||||
DOWNLOAD_BLOCK_LISTS,
|
||||
/**
|
||||
* Generates block list diffs against the previous download. The diffs consist of a stream of
|
||||
* {@link BlockOrder orders} and a stream of {@link BlockLabel labels}.
|
||||
*/
|
||||
MAKE_ORDER_AND_LABEL_DIFF,
|
||||
/** Applies the diffs to the database. */
|
||||
APPLY_ORDER_AND_LABEL_DIFF,
|
||||
/**
|
||||
* Makes a REST API call to BSA endpoint, declaring that processing starts for new orders in the
|
||||
* diffs.
|
||||
*/
|
||||
START_UPLOADING,
|
||||
/** Makes a REST API call to BSA endpoint, sending the domains that cannot be blocked. */
|
||||
UPLOAD_DOMAINS_IN_USE,
|
||||
REPORT_START_OF_ORDER_PROCESSING,
|
||||
/**
|
||||
* Makes a REST API call to BSA endpoint, uploading unblockable domains that match labels in the
|
||||
* diff.
|
||||
*/
|
||||
UPLOAD_UNBLOCKABLE_DOMAINS_FOR_NEW_ORDERS,
|
||||
/** Makes a REST API call to BSA endpoint, declaring the completion of order processing. */
|
||||
FINISH_UPLOADING,
|
||||
REPORT_END_OF_ORDER_PROCESSING,
|
||||
/** The terminal stage after processing succeeds. */
|
||||
DONE,
|
||||
/**
|
||||
|
@ -42,5 +51,5 @@ public enum DownloadStage {
|
|||
* The terminal stage indicating that the downloads are not processed because their BSA-generated
|
||||
* checksums do not match those calculated by us.
|
||||
*/
|
||||
CHECKSUMS_NOT_MATCH;
|
||||
CHECKSUMS_DO_NOT_MATCH;
|
||||
}
|
||||
|
|
229
core/src/main/java/google/registry/bsa/GcsClient.java
Normal file
229
core/src/main/java/google/registry/bsa/GcsClient.java
Normal file
|
@ -0,0 +1,229 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static com.google.common.io.BaseEncoding.base16;
|
||||
|
||||
import com.google.cloud.storage.BlobId;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import google.registry.bsa.BlockListFetcher.LazyBlockList;
|
||||
import google.registry.bsa.api.BlockLabel;
|
||||
import google.registry.bsa.api.BlockOrder;
|
||||
import google.registry.bsa.api.UnblockableDomain;
|
||||
import google.registry.bsa.api.UnblockableDomainChange;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.gcs.GcsUtils;
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.stream.Stream;
|
||||
import javax.inject.Inject;
|
||||
|
||||
/** Stores and accesses BSA-related data, including original downloads and processed data. */
|
||||
public class GcsClient {
|
||||
|
||||
// Intermediate data files:
|
||||
static final String LABELS_DIFF_FILE = "labels_diff.csv";
|
||||
static final String ORDERS_DIFF_FILE = "orders_diff.csv";
|
||||
static final String UNBLOCKABLE_DOMAINS_FILE = "unblockable_domains.csv";
|
||||
static final String REFRESHED_UNBLOCKABLE_DOMAINS_FILE = "refreshed_unblockable_domains.csv";
|
||||
|
||||
// Logged report data sent to BSA.
|
||||
static final String IN_PROGRESS_ORDERS_REPORT = "in_progress_orders.json";
|
||||
static final String COMPLETED_ORDERS_REPORT = "completed_orders.json";
|
||||
static final String ADDED_UNBLOCKABLE_DOMAINS_REPORT = "added_unblockable_domains.json";
|
||||
static final String REMOVED_UNBLOCKABLE_DOMAINS_REPORT = "removed_unblockable_domains.json";
|
||||
|
||||
private final GcsUtils gcsUtils;
|
||||
private final String bucketName;
|
||||
|
||||
private final String checksumAlgorithm;
|
||||
|
||||
@Inject
|
||||
GcsClient(
|
||||
GcsUtils gcsUtils,
|
||||
@Config("bsaGcsBucket") String bucketName,
|
||||
@Config("bsaChecksumAlgorithm") String checksumAlgorithm) {
|
||||
this.gcsUtils = gcsUtils;
|
||||
this.bucketName = bucketName;
|
||||
this.checksumAlgorithm = checksumAlgorithm;
|
||||
}
|
||||
|
||||
static String getBlockListFileName(BlockListType blockListType) {
|
||||
return blockListType.name() + ".csv";
|
||||
}
|
||||
|
||||
ImmutableMap<BlockListType, String> saveAndChecksumBlockList(
|
||||
String jobName, ImmutableList<LazyBlockList> blockLists) {
|
||||
// Downloading sequentially, since one is expected to be much smaller than the other.
|
||||
return blockLists.stream()
|
||||
.collect(
|
||||
ImmutableMap.toImmutableMap(
|
||||
LazyBlockList::getName, blockList -> saveAndChecksumBlockList(jobName, blockList)));
|
||||
}
|
||||
|
||||
private String saveAndChecksumBlockList(String jobName, LazyBlockList blockList) {
|
||||
BlobId blobId = getBlobId(jobName, getBlockListFileName(blockList.getName()));
|
||||
try (BufferedOutputStream gcsWriter =
|
||||
new BufferedOutputStream(gcsUtils.openOutputStream(blobId))) {
|
||||
MessageDigest messageDigest = MessageDigest.getInstance(checksumAlgorithm);
|
||||
blockList.consumeAll(
|
||||
(byteArray, length) -> {
|
||||
try {
|
||||
gcsWriter.write(byteArray, 0, length);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
messageDigest.update(byteArray, 0, length);
|
||||
});
|
||||
return base16().lowerCase().encode(messageDigest.digest());
|
||||
} catch (IOException | NoSuchAlgorithmException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static void writeWithNewline(BufferedWriter writer, String line) {
|
||||
try {
|
||||
writer.write(line);
|
||||
if (!line.endsWith("\n")) {
|
||||
writer.write('\n');
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
Stream<String> readBlockList(String jobName, BlockListType blockListType) {
|
||||
return readStream(getBlobId(jobName, getBlockListFileName(blockListType)));
|
||||
}
|
||||
|
||||
Stream<BlockOrder> readOrderDiffs(String jobName) {
|
||||
BlobId blobId = getBlobId(jobName, ORDERS_DIFF_FILE);
|
||||
return readStream(blobId).map(BlockOrder::deserialize);
|
||||
}
|
||||
|
||||
void writeOrderDiffs(String jobName, Stream<BlockOrder> orders) {
|
||||
BlobId blobId = getBlobId(jobName, ORDERS_DIFF_FILE);
|
||||
try (BufferedWriter gcsWriter = getWriter(blobId)) {
|
||||
orders.map(BlockOrder::serialize).forEach(line -> writeWithNewline(gcsWriter, line));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
Stream<BlockLabel> readLabelDiffs(String jobName) {
|
||||
BlobId blobId = getBlobId(jobName, LABELS_DIFF_FILE);
|
||||
return readStream(blobId).map(BlockLabel::deserialize);
|
||||
}
|
||||
|
||||
void writeLabelDiffs(String jobName, Stream<BlockLabel> labels) {
|
||||
BlobId blobId = getBlobId(jobName, LABELS_DIFF_FILE);
|
||||
try (BufferedWriter gcsWriter = getWriter(blobId)) {
|
||||
labels.map(BlockLabel::serialize).forEach(line -> writeWithNewline(gcsWriter, line));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
Stream<UnblockableDomain> readUnblockableDomains(String jobName) {
|
||||
BlobId blobId = getBlobId(jobName, UNBLOCKABLE_DOMAINS_FILE);
|
||||
return readStream(blobId).map(UnblockableDomain::deserialize);
|
||||
}
|
||||
|
||||
void writeUnblockableDomains(String jobName, Stream<UnblockableDomain> unblockables) {
|
||||
BlobId blobId = getBlobId(jobName, UNBLOCKABLE_DOMAINS_FILE);
|
||||
try (BufferedWriter gcsWriter = getWriter(blobId)) {
|
||||
unblockables
|
||||
.map(UnblockableDomain::serialize)
|
||||
.forEach(line -> writeWithNewline(gcsWriter, line));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
Stream<UnblockableDomainChange> readRefreshChanges(String jobName) {
|
||||
BlobId blobId = getBlobId(jobName, UNBLOCKABLE_DOMAINS_FILE);
|
||||
return readStream(blobId).map(UnblockableDomainChange::deserialize);
|
||||
}
|
||||
|
||||
void writeRefreshChanges(String jobName, Stream<UnblockableDomainChange> changes) {
|
||||
BlobId blobId = getBlobId(jobName, REFRESHED_UNBLOCKABLE_DOMAINS_FILE);
|
||||
try (BufferedWriter gcsWriter = getWriter(blobId)) {
|
||||
changes
|
||||
.map(UnblockableDomainChange::serialize)
|
||||
.forEach(line -> writeWithNewline(gcsWriter, line));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
void logInProgressOrderReport(String jobName, Stream<String> lines) {
|
||||
BlobId blobId = getBlobId(jobName, IN_PROGRESS_ORDERS_REPORT);
|
||||
try (BufferedWriter gcsWriter = getWriter(blobId)) {
|
||||
lines.forEach(line -> writeWithNewline(gcsWriter, line));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
void logCompletedOrderReport(String jobName, Stream<String> lines) {
|
||||
BlobId blobId = getBlobId(jobName, COMPLETED_ORDERS_REPORT);
|
||||
try (BufferedWriter gcsWriter = getWriter(blobId)) {
|
||||
lines.forEach(line -> writeWithNewline(gcsWriter, line));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
void logAddedUnblockableDomainsReport(String jobName, Stream<String> lines) {
|
||||
BlobId blobId = getBlobId(jobName, ADDED_UNBLOCKABLE_DOMAINS_REPORT);
|
||||
try (BufferedWriter gcsWriter = getWriter(blobId)) {
|
||||
lines.forEach(line -> writeWithNewline(gcsWriter, line));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
void logRemovedUnblockableDomainsReport(String jobName, Stream<String> lines) {
|
||||
BlobId blobId = getBlobId(jobName, REMOVED_UNBLOCKABLE_DOMAINS_REPORT);
|
||||
try (BufferedWriter gcsWriter = getWriter(blobId)) {
|
||||
lines.forEach(line -> writeWithNewline(gcsWriter, line));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
BlobId getBlobId(String folder, String name) {
|
||||
return BlobId.of(bucketName, String.format("%s/%s", folder, name));
|
||||
}
|
||||
|
||||
Stream<String> readStream(BlobId blobId) {
|
||||
return new BufferedReader(
|
||||
new InputStreamReader(gcsUtils.openInputStream(blobId), StandardCharsets.UTF_8))
|
||||
.lines();
|
||||
}
|
||||
|
||||
BufferedWriter getWriter(BlobId blobId) {
|
||||
return new BufferedWriter(
|
||||
new OutputStreamWriter(gcsUtils.openOutputStream(blobId), StandardCharsets.UTF_8));
|
||||
}
|
||||
}
|
|
@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.ImmutableMultimap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.collect.Sets.SetView;
|
||||
import google.registry.model.tld.Tld;
|
||||
import google.registry.model.tld.Tld.TldType;
|
||||
import google.registry.model.tld.Tlds;
|
||||
|
@ -76,8 +75,8 @@ public class IdnChecker {
|
|||
*
|
||||
* @param idnTables String names of {@link IdnTableEnum} values
|
||||
*/
|
||||
public SetView<Tld> getForbiddingTlds(ImmutableSet<String> idnTables) {
|
||||
return Sets.difference(allTlds, getSupportingTlds(idnTables));
|
||||
public ImmutableSet<Tld> getForbiddingTlds(ImmutableSet<String> idnTables) {
|
||||
return Sets.difference(allTlds, getSupportingTlds(idnTables)).immutableCopy();
|
||||
}
|
||||
|
||||
private static ImmutableMap<IdnTableEnum, ImmutableSet<Tld>> getIdnToTldMap(DateTime now) {
|
||||
|
|
30
core/src/main/java/google/registry/bsa/RefreshStage.java
Normal file
30
core/src/main/java/google/registry/bsa/RefreshStage.java
Normal file
|
@ -0,0 +1,30 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
public enum RefreshStage {
|
||||
/**
|
||||
* Checks for stale unblockable domains. The output is a stream of {@link
|
||||
* google.registry.bsa.api.UnblockableDomainChange} objects that describe the stale domains.
|
||||
*/
|
||||
CHECK_FOR_CHANGES,
|
||||
/** Fixes the stale domains in the database. */
|
||||
APPLY_CHANGES,
|
||||
/** Reports the unblockable domains to be removed to BSA. */
|
||||
UPLOAD_REMOVALS,
|
||||
/** Reports the newly found unblockable domains to BSA. */
|
||||
UPLOAD_ADDITIONS,
|
||||
DONE;
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||
import static google.registry.bsa.BsaStringUtils.DOMAIN_JOINER;
|
||||
import static google.registry.flows.domain.DomainFlowUtils.isReserved;
|
||||
import static google.registry.model.tld.Tlds.findTldForName;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.net.InternetDomainName;
|
||||
import google.registry.model.tld.Tld;
|
||||
import google.registry.model.tld.Tld.TldState;
|
||||
import google.registry.model.tld.Tld.TldType;
|
||||
import google.registry.model.tld.Tlds;
|
||||
import google.registry.model.tld.label.ReservedList;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Stream;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/**
|
||||
* Utility for looking up reserved domain names.
|
||||
*
|
||||
* <p>This utility is only concerned with reserved domains that can be created (with appropriate
|
||||
* tokens).
|
||||
*/
|
||||
public final class ReservedDomainsUtils {
|
||||
|
||||
private ReservedDomainsUtils() {}
|
||||
|
||||
public static Stream<String> getAllReservedNames(DateTime now) {
|
||||
return Tlds.getTldEntitiesOfType(TldType.REAL).stream()
|
||||
.filter(tld -> Tld.isEnrolledWithBsa(tld, now))
|
||||
.map(tld -> getAllReservedDomainsInTld(tld, now))
|
||||
.flatMap(ImmutableSet::stream);
|
||||
}
|
||||
|
||||
/** Returns */
|
||||
static ImmutableSet<String> getAllReservedDomainsInTld(Tld tld, DateTime now) {
|
||||
return tld.getReservedListNames().stream()
|
||||
.map(ReservedList::get)
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get)
|
||||
.map(ReservedList::getReservedListEntries)
|
||||
.map(Map::keySet)
|
||||
.flatMap(Set::stream)
|
||||
.map(label -> DOMAIN_JOINER.join(label, tld.getTldStr()))
|
||||
.filter(domain -> isReservedDomain(domain, now))
|
||||
.collect(toImmutableSet());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if {@code domain} is a reserved name that can be registered right now (e.g.,
|
||||
* during sunrise or with allocation token), therefore unblockable.
|
||||
*/
|
||||
public static boolean isReservedDomain(String domain, DateTime now) {
|
||||
Optional<InternetDomainName> tldStr = findTldForName(InternetDomainName.from(domain));
|
||||
verify(tldStr.isPresent(), "Tld for domain [%s] unexpectedly missing.", domain);
|
||||
Tld tld = Tld.get(tldStr.get().toString());
|
||||
return isReserved(
|
||||
InternetDomainName.from(domain),
|
||||
Objects.equals(tld.getTldState(now), TldState.START_DATE_SUNRISE));
|
||||
}
|
||||
}
|
64
core/src/main/java/google/registry/bsa/api/BlockLabel.java
Normal file
64
core/src/main/java/google/registry/bsa/api/BlockLabel.java
Normal file
|
@ -0,0 +1,64 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A BSA label to block. New domains with matching second-level domain (SLD) will be denied
|
||||
* registration in TLDs enrolled with BSA.
|
||||
*/
|
||||
@AutoValue
|
||||
public abstract class BlockLabel {
|
||||
|
||||
static final Joiner JOINER = Joiner.on(',');
|
||||
static final Splitter SPLITTER = Splitter.on(',').trimResults();
|
||||
|
||||
public abstract String label();
|
||||
|
||||
public abstract LabelType labelType();
|
||||
|
||||
public abstract ImmutableSet<String> idnTables();
|
||||
|
||||
public String serialize() {
|
||||
return JOINER.join(label(), labelType().name(), idnTables().stream().sorted().toArray());
|
||||
}
|
||||
|
||||
public static BlockLabel deserialize(String text) {
|
||||
List<String> items = SPLITTER.splitToList(text);
|
||||
try {
|
||||
return of(
|
||||
items.get(0),
|
||||
LabelType.valueOf(items.get(1)),
|
||||
ImmutableSet.copyOf(items.subList(2, items.size())));
|
||||
} catch (NumberFormatException ne) {
|
||||
throw new IllegalArgumentException(text);
|
||||
}
|
||||
}
|
||||
|
||||
public static BlockLabel of(String label, LabelType type, ImmutableSet<String> idnTables) {
|
||||
return new AutoValue_BlockLabel(label, type, idnTables);
|
||||
}
|
||||
|
||||
public enum LabelType {
|
||||
CREATE,
|
||||
NEW_ORDER_ASSOCIATION,
|
||||
DELETE;
|
||||
}
|
||||
}
|
57
core/src/main/java/google/registry/bsa/api/BlockOrder.java
Normal file
57
core/src/main/java/google/registry/bsa/api/BlockOrder.java
Normal file
|
@ -0,0 +1,57 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Splitter;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A BSA order, which are needed when communicating with the BSA API while processing downloaded
|
||||
* block lists.
|
||||
*/
|
||||
@AutoValue
|
||||
public abstract class BlockOrder {
|
||||
|
||||
public abstract long orderId();
|
||||
|
||||
public abstract OrderType orderType();
|
||||
|
||||
static final Joiner JOINER = Joiner.on(',');
|
||||
static final Splitter SPLITTER = Splitter.on(',');
|
||||
|
||||
public String serialize() {
|
||||
return JOINER.join(orderId(), orderType().name());
|
||||
}
|
||||
|
||||
public static BlockOrder deserialize(String text) {
|
||||
List<String> items = SPLITTER.splitToList(text);
|
||||
try {
|
||||
return of(Long.valueOf(items.get(0)), OrderType.valueOf(items.get(1)));
|
||||
} catch (NumberFormatException ne) {
|
||||
throw new IllegalArgumentException(text);
|
||||
}
|
||||
}
|
||||
|
||||
public static BlockOrder of(long orderId, OrderType orderType) {
|
||||
return new AutoValue_BlockOrder(orderId, orderType);
|
||||
}
|
||||
|
||||
public enum OrderType {
|
||||
CREATE,
|
||||
DELETE;
|
||||
}
|
||||
}
|
160
core/src/main/java/google/registry/bsa/api/BsaCredential.java
Normal file
160
core/src/main/java/google/registry/bsa/api/BsaCredential.java
Normal file
|
@ -0,0 +1,160 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static google.registry.request.UrlConnectionUtils.getResponseBytes;
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
|
||||
import com.google.api.client.http.HttpMethods;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.google.gson.Gson;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.keyring.api.Keyring;
|
||||
import google.registry.request.UrlConnectionService;
|
||||
import google.registry.request.UrlConnectionUtils;
|
||||
import google.registry.util.Clock;
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.security.GeneralSecurityException;
|
||||
import java.util.Map;
|
||||
import javax.annotation.Nullable;
|
||||
import javax.annotation.concurrent.ThreadSafe;
|
||||
import javax.inject.Inject;
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.Instant;
|
||||
|
||||
/**
|
||||
* A credential for accessing the BSA API.
|
||||
*
|
||||
* <p>Fetches on-demand an auth token from BSA's auth http endpoint and caches it for repeated use
|
||||
* until the token expires (expiry set by BSA and recorded in the configuration file). An expired
|
||||
* token is refreshed only when requested. Token refreshing is blocking but thread-safe.
|
||||
*
|
||||
* <p>The token-fetching request authenticates itself with an API key, which is stored in the Secret
|
||||
* Manager.
|
||||
*/
|
||||
@ThreadSafe
|
||||
public class BsaCredential {
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
/** Content type of the auth http request. */
|
||||
private static final String CONTENT_TYPE = "application/x-www-form-urlencoded";
|
||||
/** Template of the auth http request's payload. User must provide an API key. */
|
||||
private static final String AUTH_REQ_BODY_TEMPLATE = "apiKey=%s&space=BSA";
|
||||
/** The variable name for the auth token in the returned json response. */
|
||||
public static final String ID_TOKEN = "id_token";
|
||||
|
||||
private final UrlConnectionService urlConnectionService;
|
||||
|
||||
private final String authUrl;
|
||||
|
||||
private final Duration authTokenExpiry;
|
||||
|
||||
private final Keyring keyring;
|
||||
|
||||
private final Clock clock;
|
||||
|
||||
@Nullable private String authToken;
|
||||
private Instant lastRefreshTime;
|
||||
|
||||
@Inject
|
||||
BsaCredential(
|
||||
UrlConnectionService urlConnectionService,
|
||||
@Config("bsaAuthUrl") String authUrl,
|
||||
@Config("bsaAuthTokenExpiry") Duration authTokenExpiry,
|
||||
Keyring keyring,
|
||||
Clock clock) {
|
||||
this.urlConnectionService = urlConnectionService;
|
||||
this.authUrl = authUrl;
|
||||
this.authTokenExpiry = authTokenExpiry;
|
||||
this.keyring = keyring;
|
||||
this.clock = clock;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the auth token for accessing the BSA API.
|
||||
*
|
||||
* <p>This method refreshes the token if it is expired, and is thread-safe..
|
||||
*/
|
||||
public String getAuthToken() {
|
||||
try {
|
||||
ensureAuthTokenValid();
|
||||
} catch (IOException e) {
|
||||
throw new BsaException(e, /* retriable= */ true);
|
||||
} catch (GeneralSecurityException e) {
|
||||
throw new BsaException(e, /* retriable= */ false);
|
||||
}
|
||||
return this.authToken;
|
||||
}
|
||||
|
||||
private void ensureAuthTokenValid() throws IOException, GeneralSecurityException {
|
||||
Instant now = Instant.ofEpochMilli(clock.nowUtc().getMillis());
|
||||
if (authToken != null && lastRefreshTime.plus(authTokenExpiry).isAfter(now)) {
|
||||
logger.atInfo().log("AuthToken still valid, reusing.");
|
||||
return;
|
||||
}
|
||||
synchronized (this) {
|
||||
authToken = fetchNewAuthToken();
|
||||
lastRefreshTime = now;
|
||||
logger.atInfo().log("AuthToken refreshed at %s.", now);
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
String fetchNewAuthToken() throws IOException, GeneralSecurityException {
|
||||
String payload = String.format(AUTH_REQ_BODY_TEMPLATE, keyring.getBsaApiKey());
|
||||
URL url = new URL(authUrl);
|
||||
logger.atInfo().log("Fetching auth token from %s", url);
|
||||
HttpsURLConnection connection = null;
|
||||
try {
|
||||
connection = (HttpsURLConnection) urlConnectionService.createConnection(url);
|
||||
connection.setRequestMethod(HttpMethods.POST);
|
||||
UrlConnectionUtils.setPayload(connection, payload.getBytes(UTF_8), CONTENT_TYPE);
|
||||
int code = connection.getResponseCode();
|
||||
if (code != SC_OK) {
|
||||
String errorDetails;
|
||||
try {
|
||||
errorDetails = new String(getResponseBytes(connection), UTF_8);
|
||||
} catch (Exception e) {
|
||||
errorDetails = "Failed to retrieve error message: " + e.getMessage();
|
||||
}
|
||||
throw new BsaException(
|
||||
String.format(
|
||||
"Status code: [%s], error: [%s], details: [%s]",
|
||||
code, connection.getResponseMessage(), errorDetails),
|
||||
/* retriable= */ true);
|
||||
}
|
||||
// TODO: catch json syntax exception
|
||||
@SuppressWarnings("unchecked")
|
||||
String idToken =
|
||||
new Gson()
|
||||
.fromJson(new String(getResponseBytes(connection), UTF_8), Map.class)
|
||||
.getOrDefault(ID_TOKEN, "")
|
||||
.toString();
|
||||
if (idToken.isEmpty()) {
|
||||
throw new BsaException("Response missing ID token", /* retriable= */ false);
|
||||
}
|
||||
return idToken;
|
||||
} finally {
|
||||
if (connection != null) {
|
||||
connection.disconnect();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
34
core/src/main/java/google/registry/bsa/api/BsaException.java
Normal file
34
core/src/main/java/google/registry/bsa/api/BsaException.java
Normal file
|
@ -0,0 +1,34 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
public class BsaException extends RuntimeException {
|
||||
|
||||
private final boolean retriable;
|
||||
|
||||
public BsaException(Throwable cause, boolean retriable) {
|
||||
super(cause);
|
||||
this.retriable = retriable;
|
||||
}
|
||||
|
||||
public BsaException(String message, boolean retriable) {
|
||||
super(message);
|
||||
this.retriable = retriable;
|
||||
}
|
||||
|
||||
public boolean isRetriable() {
|
||||
return this.retriable;
|
||||
}
|
||||
}
|
127
core/src/main/java/google/registry/bsa/api/BsaReportSender.java
Normal file
127
core/src/main/java/google/registry/bsa/api/BsaReportSender.java
Normal file
|
@ -0,0 +1,127 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_ACCEPTED;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
|
||||
import com.google.api.client.http.HttpMethods;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.net.MediaType;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.request.UrlConnectionService;
|
||||
import google.registry.request.UrlConnectionUtils;
|
||||
import google.registry.util.Retrier;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URL;
|
||||
import java.security.GeneralSecurityException;
|
||||
import javax.inject.Inject;
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
|
||||
/**
|
||||
* Sends order processing reports to BSA.
|
||||
*
|
||||
* <p>Senders are responsible for keeping payloads at reasonable sizes.
|
||||
*/
|
||||
public class BsaReportSender {
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
private static final MediaType CONTENT_TYPE = MediaType.JSON_UTF_8;
|
||||
|
||||
private final UrlConnectionService urlConnectionService;
|
||||
private final BsaCredential credential;
|
||||
private final String orderStatusUrl;
|
||||
private final String addUnblockableDomainsUrl;
|
||||
private final String removeUnblockableDomainsUrl;
|
||||
|
||||
private final Retrier retrier;
|
||||
|
||||
@Inject
|
||||
BsaReportSender(
|
||||
UrlConnectionService urlConnectionService,
|
||||
BsaCredential credential,
|
||||
@Config("bsaOrderStatusUrl") String orderStatusUrl,
|
||||
@Config("bsaAddUnblockableDomainsUrl") String addUnblockableDomainsUrl,
|
||||
@Config("bsaRemoveUnblockableDomainsUrl") String removeUnblockableDomainsUrl,
|
||||
Retrier retrier) {
|
||||
this.urlConnectionService = urlConnectionService;
|
||||
this.credential = credential;
|
||||
this.orderStatusUrl = orderStatusUrl;
|
||||
this.addUnblockableDomainsUrl = addUnblockableDomainsUrl;
|
||||
this.removeUnblockableDomainsUrl = removeUnblockableDomainsUrl;
|
||||
this.retrier = retrier;
|
||||
}
|
||||
|
||||
public void sendOrderStatusReport(String payload) {
|
||||
retrier.callWithRetry(
|
||||
() -> trySendData(this.orderStatusUrl, payload),
|
||||
e -> e instanceof BsaException && ((BsaException) e).isRetriable());
|
||||
}
|
||||
|
||||
public void addUnblockableDomainsUpdates(String payload) {
|
||||
retrier.callWithRetry(
|
||||
() -> trySendData(this.addUnblockableDomainsUrl, payload),
|
||||
e -> e instanceof BsaException && ((BsaException) e).isRetriable());
|
||||
}
|
||||
|
||||
public void removeUnblockableDomainsUpdates(String payload) {
|
||||
retrier.callWithRetry(
|
||||
() -> trySendData(this.removeUnblockableDomainsUrl, payload),
|
||||
e -> e instanceof BsaException && ((BsaException) e).isRetriable());
|
||||
}
|
||||
|
||||
Void trySendData(String urlString, String payload) {
|
||||
try {
|
||||
URL url = new URL(urlString);
|
||||
HttpsURLConnection connection =
|
||||
(HttpsURLConnection) urlConnectionService.createConnection(url);
|
||||
connection.setRequestMethod(HttpMethods.POST);
|
||||
connection.setRequestProperty("Authorization", "Bearer " + credential.getAuthToken());
|
||||
UrlConnectionUtils.setPayload(connection, payload.getBytes(UTF_8), CONTENT_TYPE.toString());
|
||||
int code = connection.getResponseCode();
|
||||
if (code != SC_OK && code != SC_ACCEPTED) {
|
||||
String errorDetails = "";
|
||||
try (InputStream errorStream = connection.getErrorStream()) {
|
||||
errorDetails = new String(ByteStreams.toByteArray(errorStream), UTF_8);
|
||||
} catch (NullPointerException e) {
|
||||
// No error message.
|
||||
} catch (Exception e) {
|
||||
errorDetails = "Failed to retrieve error message: " + e.getMessage();
|
||||
}
|
||||
// TODO(b/318404541): sanitize errorDetails to prevent log injection attack.
|
||||
throw new BsaException(
|
||||
String.format(
|
||||
"Status code: [%s], error: [%s], details: [%s]",
|
||||
code, connection.getResponseMessage(), errorDetails),
|
||||
/* retriable= */ true);
|
||||
}
|
||||
try (InputStream errorStream = connection.getInputStream()) {
|
||||
String responseMessage = new String(ByteStreams.toByteArray(errorStream), UTF_8);
|
||||
logger.atInfo().log("Received response: [%s]", responseMessage);
|
||||
} catch (Exception e) {
|
||||
logger.atInfo().withCause(e).log("Failed to retrieve response message.");
|
||||
}
|
||||
return null;
|
||||
} catch (IOException e) {
|
||||
throw new BsaException(e, /* retriable= */ true);
|
||||
} catch (GeneralSecurityException e) {
|
||||
throw new BsaException(e, /* retriable= */ false);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static com.google.common.collect.Maps.newTreeMap;
|
||||
import static com.google.common.collect.Multimaps.newListMultimap;
|
||||
import static com.google.common.collect.Multimaps.toMultimap;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableMultimap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.gson.Gson;
|
||||
import com.google.gson.GsonBuilder;
|
||||
import google.registry.bsa.api.BlockOrder.OrderType;
|
||||
import java.util.Locale;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/** Helpers for generating {@link BlockOrder} and {@link UnblockableDomain} reports. */
|
||||
public final class JsonSerializations {
|
||||
|
||||
private static final Gson GSON = new GsonBuilder().setPrettyPrinting().create();
|
||||
|
||||
private JsonSerializations() {}
|
||||
|
||||
public static Optional<String> toInProgressOrdersReport(Stream<BlockOrder> orders) {
|
||||
ImmutableList<ImmutableMap<String, Object>> maps =
|
||||
orders.map(JsonSerializations::asInProgressOrder).collect(toImmutableList());
|
||||
if (maps.isEmpty()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(GSON.toJson(maps));
|
||||
}
|
||||
|
||||
public static Optional<String> toCompletedOrdersReport(Stream<BlockOrder> orders) {
|
||||
ImmutableList<ImmutableMap<String, Object>> maps =
|
||||
orders.map(JsonSerializations::asCompletedOrder).collect(toImmutableList());
|
||||
if (maps.isEmpty()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(GSON.toJson(maps));
|
||||
}
|
||||
|
||||
public static Optional<String> toUnblockableDomainsReport(Stream<UnblockableDomain> domains) {
|
||||
ImmutableMultimap<String, String> reasonToNames =
|
||||
ImmutableMultimap.copyOf(
|
||||
domains.collect(
|
||||
toMultimap(
|
||||
domain -> domain.reason().name().toLowerCase(Locale.ROOT),
|
||||
UnblockableDomain::domainName,
|
||||
() -> newListMultimap(newTreeMap(), Lists::newArrayList))));
|
||||
|
||||
if (reasonToNames.isEmpty()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(GSON.toJson(reasonToNames.asMap()));
|
||||
}
|
||||
|
||||
public static Optional<String> toUnblockableDomainsRemovalReport(Stream<String> domainNames) {
|
||||
ImmutableList<String> domainsList = domainNames.collect(toImmutableList());
|
||||
if (domainsList.isEmpty()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(GSON.toJson(domainsList));
|
||||
}
|
||||
|
||||
private static ImmutableMap<String, Object> asInProgressOrder(BlockOrder order) {
|
||||
String status =
|
||||
order.orderType().equals(OrderType.CREATE) ? "ActivationInProgress" : "ReleaseInProgress";
|
||||
return ImmutableMap.of("blockOrderId", order.orderId(), "status", status);
|
||||
}
|
||||
|
||||
private static ImmutableMap<String, Object> asCompletedOrder(BlockOrder order) {
|
||||
String status = order.orderType().equals(OrderType.CREATE) ? "Active" : "Closed";
|
||||
return ImmutableMap.of("blockOrderId", order.orderId(), "status", status);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,58 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static google.registry.bsa.BsaStringUtils.DOMAIN_JOINER;
|
||||
import static google.registry.bsa.BsaStringUtils.PROPERTY_JOINER;
|
||||
import static google.registry.bsa.BsaStringUtils.PROPERTY_SPLITTER;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A domain name whose second-level domain (SLD) matches a BSA label but is not blocked. It may be
|
||||
* already registered, or on the TLD's reserve list.
|
||||
*/
|
||||
// TODO(1/15/2024): rename to UnblockableDomain.
|
||||
@AutoValue
|
||||
public abstract class UnblockableDomain {
|
||||
abstract String domainName();
|
||||
|
||||
abstract Reason reason();
|
||||
|
||||
/** Reasons why a valid domain name cannot be blocked. */
|
||||
public enum Reason {
|
||||
REGISTERED,
|
||||
RESERVED,
|
||||
INVALID;
|
||||
}
|
||||
|
||||
public String serialize() {
|
||||
return PROPERTY_JOINER.join(domainName(), reason().name());
|
||||
}
|
||||
|
||||
public static UnblockableDomain deserialize(String text) {
|
||||
List<String> items = PROPERTY_SPLITTER.splitToList(text);
|
||||
return of(items.get(0), Reason.valueOf(items.get(1)));
|
||||
}
|
||||
|
||||
public static UnblockableDomain of(String domainName, Reason reason) {
|
||||
return new AutoValue_UnblockableDomain(domainName, reason);
|
||||
}
|
||||
|
||||
public static UnblockableDomain of(String label, String tld, Reason reason) {
|
||||
return of(DOMAIN_JOINER.join(label, tld), reason);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,99 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static google.registry.bsa.BsaStringUtils.PROPERTY_JOINER;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.auto.value.extension.memoized.Memoized;
|
||||
import google.registry.bsa.BsaStringUtils;
|
||||
import google.registry.bsa.api.UnblockableDomain.Reason;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
/** Change record of an {@link UnblockableDomain}. */
|
||||
@AutoValue
|
||||
public abstract class UnblockableDomainChange {
|
||||
|
||||
/**
|
||||
* The text used in place of an empty {@link #newReason()} when an instance is serialized to
|
||||
* string.
|
||||
*
|
||||
* <p>This value helps manual inspection of the change files, making it easier to `grep` for
|
||||
* deletions in BSA reports.
|
||||
*/
|
||||
private static final String DELETE_REASON_PLACEHOLDER = "IS_DELETE";
|
||||
|
||||
abstract UnblockableDomain unblockable();
|
||||
|
||||
abstract Optional<Reason> newReason();
|
||||
|
||||
public String domainName() {
|
||||
return unblockable().domainName();
|
||||
}
|
||||
|
||||
@Memoized
|
||||
public UnblockableDomain newValue() {
|
||||
verify(newReason().isPresent(), "Removed unblockable does not have new value.");
|
||||
return UnblockableDomain.of(unblockable().domainName(), newReason().get());
|
||||
}
|
||||
|
||||
public boolean AddOrChange() {
|
||||
return newReason().isPresent();
|
||||
}
|
||||
|
||||
public boolean isDelete() {
|
||||
return !this.AddOrChange();
|
||||
}
|
||||
|
||||
public boolean isNew() {
|
||||
return newReason().filter(unblockable().reason()::equals).isPresent();
|
||||
}
|
||||
|
||||
public String serialize() {
|
||||
return PROPERTY_JOINER.join(
|
||||
unblockable().domainName(),
|
||||
unblockable().reason(),
|
||||
newReason().map(Reason::name).orElse(DELETE_REASON_PLACEHOLDER));
|
||||
}
|
||||
|
||||
public static UnblockableDomainChange deserialize(String text) {
|
||||
List<String> items = BsaStringUtils.PROPERTY_SPLITTER.splitToList(text);
|
||||
return of(
|
||||
UnblockableDomain.of(items.get(0), Reason.valueOf(items.get(1))),
|
||||
Objects.equals(items.get(2), DELETE_REASON_PLACEHOLDER)
|
||||
? Optional.empty()
|
||||
: Optional.of(Reason.valueOf(items.get(2))));
|
||||
}
|
||||
|
||||
public static UnblockableDomainChange ofNew(UnblockableDomain unblockable) {
|
||||
return of(unblockable, Optional.of(unblockable.reason()));
|
||||
}
|
||||
|
||||
public static UnblockableDomainChange ofDeleted(UnblockableDomain unblockable) {
|
||||
return of(unblockable, Optional.empty());
|
||||
}
|
||||
|
||||
public static UnblockableDomainChange ofChanged(UnblockableDomain unblockable, Reason newReason) {
|
||||
return of(unblockable, Optional.of(newReason));
|
||||
}
|
||||
|
||||
private static UnblockableDomainChange of(
|
||||
UnblockableDomain unblockable, Optional<Reason> newReason) {
|
||||
return new AutoValue_UnblockableDomainChange(unblockable, newReason);
|
||||
}
|
||||
}
|
|
@ -1,102 +0,0 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import google.registry.bsa.persistence.BsaDomainInUse.BsaDomainInUseId;
|
||||
import google.registry.model.CreateAutoTimestamp;
|
||||
import google.registry.persistence.VKey;
|
||||
import java.io.Serializable;
|
||||
import javax.persistence.Column;
|
||||
import javax.persistence.Entity;
|
||||
import javax.persistence.EnumType;
|
||||
import javax.persistence.Enumerated;
|
||||
import javax.persistence.Id;
|
||||
import javax.persistence.IdClass;
|
||||
|
||||
/** A domain matching a BSA label but is in use (registered or reserved), so cannot be blocked. */
|
||||
@Entity
|
||||
@IdClass(BsaDomainInUseId.class)
|
||||
public class BsaDomainInUse {
|
||||
@Id String label;
|
||||
@Id String tld;
|
||||
|
||||
@Column(nullable = false)
|
||||
@Enumerated(EnumType.STRING)
|
||||
Reason reason;
|
||||
|
||||
/**
|
||||
* Creation time of this record, which is the most recent time when the domain was detected to be
|
||||
* in use wrt BSA. It may be during the processing of a download, or during some other job that
|
||||
* refreshes the state.
|
||||
*
|
||||
* <p>This field is for information only.
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
@Column(nullable = false)
|
||||
CreateAutoTimestamp createTime = CreateAutoTimestamp.create(null);
|
||||
|
||||
// For Hibernate
|
||||
BsaDomainInUse() {}
|
||||
|
||||
public BsaDomainInUse(String label, String tld, Reason reason) {
|
||||
this.label = label;
|
||||
this.tld = tld;
|
||||
this.reason = reason;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof BsaDomainInUse)) {
|
||||
return false;
|
||||
}
|
||||
BsaDomainInUse that = (BsaDomainInUse) o;
|
||||
return Objects.equal(label, that.label)
|
||||
&& Objects.equal(tld, that.tld)
|
||||
&& reason == that.reason
|
||||
&& Objects.equal(createTime, that.createTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(label, tld, reason, createTime);
|
||||
}
|
||||
|
||||
enum Reason {
|
||||
REGISTERED,
|
||||
RESERVED;
|
||||
}
|
||||
|
||||
static class BsaDomainInUseId implements Serializable {
|
||||
|
||||
private String label;
|
||||
private String tld;
|
||||
|
||||
// For Hibernate
|
||||
BsaDomainInUseId() {}
|
||||
|
||||
BsaDomainInUseId(String label, String tld) {
|
||||
this.label = label;
|
||||
this.tld = tld;
|
||||
}
|
||||
}
|
||||
|
||||
static VKey<BsaDomainInUse> vKey(String label, String tld) {
|
||||
return VKey.create(BsaDomainInUse.class, new BsaDomainInUseId(label, tld));
|
||||
}
|
||||
}
|
|
@ -14,9 +14,11 @@
|
|||
|
||||
package google.registry.bsa.persistence;
|
||||
|
||||
import static google.registry.bsa.persistence.BsaDomainRefresh.Stage.MAKE_DIFF;
|
||||
import static google.registry.bsa.RefreshStage.CHECK_FOR_CHANGES;
|
||||
import static google.registry.bsa.RefreshStage.DONE;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import google.registry.bsa.RefreshStage;
|
||||
import google.registry.model.CreateAutoTimestamp;
|
||||
import google.registry.model.UpdateAutoTimestamp;
|
||||
import google.registry.persistence.VKey;
|
||||
|
@ -37,7 +39,7 @@ import org.joda.time.DateTime;
|
|||
* change status when the IDN tables change, and will be handled by a separate tool when it happens.
|
||||
*/
|
||||
@Entity
|
||||
public class BsaDomainRefresh {
|
||||
class BsaDomainRefresh {
|
||||
|
||||
@Id
|
||||
@GeneratedValue(strategy = GenerationType.IDENTITY)
|
||||
|
@ -51,7 +53,7 @@ public class BsaDomainRefresh {
|
|||
|
||||
@Column(nullable = false)
|
||||
@Enumerated(EnumType.STRING)
|
||||
Stage stage = MAKE_DIFF;
|
||||
RefreshStage stage = CHECK_FOR_CHANGES;
|
||||
|
||||
BsaDomainRefresh() {}
|
||||
|
||||
|
@ -67,21 +69,25 @@ public class BsaDomainRefresh {
|
|||
* Returns the starting time of this job as a string, which can be used as folder name on GCS when
|
||||
* storing download data.
|
||||
*/
|
||||
public String getJobName() {
|
||||
return "refresh-" + getCreationTime().toString();
|
||||
String getJobName() {
|
||||
return getCreationTime().toString() + "-refresh";
|
||||
}
|
||||
|
||||
public Stage getStage() {
|
||||
boolean isDone() {
|
||||
return java.util.Objects.equals(stage, DONE);
|
||||
}
|
||||
|
||||
RefreshStage getStage() {
|
||||
return this.stage;
|
||||
}
|
||||
|
||||
BsaDomainRefresh setStage(Stage stage) {
|
||||
this.stage = stage;
|
||||
BsaDomainRefresh setStage(RefreshStage refreshStage) {
|
||||
this.stage = refreshStage;
|
||||
return this;
|
||||
}
|
||||
|
||||
VKey<BsaDomainRefresh> vKey() {
|
||||
return vKey(this);
|
||||
return vKey(jobId);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -104,14 +110,7 @@ public class BsaDomainRefresh {
|
|||
return Objects.hashCode(jobId, creationTime, updateTime, stage);
|
||||
}
|
||||
|
||||
static VKey vKey(BsaDomainRefresh bsaDomainRefresh) {
|
||||
return VKey.create(BsaDomainRefresh.class, bsaDomainRefresh.jobId);
|
||||
}
|
||||
|
||||
enum Stage {
|
||||
MAKE_DIFF,
|
||||
APPLY_DIFF,
|
||||
REPORT_REMOVALS,
|
||||
REPORT_ADDITIONS;
|
||||
static VKey<BsaDomainRefresh> vKey(long jobId) {
|
||||
return VKey.create(BsaDomainRefresh.class, jobId);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -15,18 +15,20 @@
|
|||
package google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.collect.ImmutableMap.toImmutableMap;
|
||||
import static google.registry.bsa.DownloadStage.DOWNLOAD;
|
||||
import static google.registry.bsa.DownloadStage.DONE;
|
||||
import static google.registry.bsa.DownloadStage.DOWNLOAD_BLOCK_LISTS;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSortedMap;
|
||||
import google.registry.bsa.BlockList;
|
||||
import google.registry.bsa.BlockListType;
|
||||
import google.registry.bsa.DownloadStage;
|
||||
import google.registry.model.CreateAutoTimestamp;
|
||||
import google.registry.model.UpdateAutoTimestamp;
|
||||
import google.registry.persistence.VKey;
|
||||
import java.util.Locale;
|
||||
import javax.persistence.Column;
|
||||
import javax.persistence.Entity;
|
||||
import javax.persistence.EnumType;
|
||||
|
@ -41,7 +43,7 @@ import org.joda.time.DateTime;
|
|||
/** Records of ongoing and completed download jobs. */
|
||||
@Entity
|
||||
@Table(indexes = {@Index(columnList = "creationTime")})
|
||||
public class BsaDownload {
|
||||
class BsaDownload {
|
||||
|
||||
private static final Joiner CSV_JOINER = Joiner.on(',');
|
||||
private static final Splitter CSV_SPLITTER = Splitter.on(',');
|
||||
|
@ -61,7 +63,7 @@ public class BsaDownload {
|
|||
|
||||
@Column(nullable = false)
|
||||
@Enumerated(EnumType.STRING)
|
||||
DownloadStage stage = DOWNLOAD;
|
||||
DownloadStage stage = DOWNLOAD_BLOCK_LISTS;
|
||||
|
||||
BsaDownload() {}
|
||||
|
||||
|
@ -74,14 +76,21 @@ public class BsaDownload {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the starting time of this job as a string, which can be used as folder name on GCS when
|
||||
* storing download data.
|
||||
* Returns a unique name of the job.
|
||||
*
|
||||
* <p>The returned value should be a valid GCS folder name, consisting of only lower case
|
||||
* alphanumerics, underscore, hyphen and dot.
|
||||
*/
|
||||
public String getJobName() {
|
||||
return getCreationTime().toString();
|
||||
String getJobName() {
|
||||
// Return a value based on job start time, which is unique.
|
||||
return getCreationTime().toString().toLowerCase(Locale.ROOT).replace(":", "");
|
||||
}
|
||||
|
||||
public DownloadStage getStage() {
|
||||
boolean isDone() {
|
||||
return java.util.Objects.equals(stage, DONE);
|
||||
}
|
||||
|
||||
DownloadStage getStage() {
|
||||
return this.stage;
|
||||
}
|
||||
|
||||
|
@ -90,19 +99,20 @@ public class BsaDownload {
|
|||
return this;
|
||||
}
|
||||
|
||||
BsaDownload setChecksums(ImmutableMap<BlockList, String> checksums) {
|
||||
BsaDownload setChecksums(ImmutableMap<BlockListType, String> checksums) {
|
||||
blockListChecksums =
|
||||
CSV_JOINER.withKeyValueSeparator("=").join(ImmutableSortedMap.copyOf(checksums));
|
||||
return this;
|
||||
}
|
||||
|
||||
ImmutableMap<BlockList, String> getChecksums() {
|
||||
ImmutableMap<BlockListType, String> getChecksums() {
|
||||
if (blockListChecksums.isEmpty()) {
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
return CSV_SPLITTER.withKeyValueSeparator('=').split(blockListChecksums).entrySet().stream()
|
||||
.collect(
|
||||
toImmutableMap(entry -> BlockList.valueOf(entry.getKey()), entry -> entry.getValue()));
|
||||
toImmutableMap(
|
||||
entry -> BlockListType.valueOf(entry.getKey()), entry -> entry.getValue()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -44,7 +44,7 @@ final class BsaLabel {
|
|||
DateTime creationTime;
|
||||
|
||||
// For Hibernate.
|
||||
BsaLabel() {}
|
||||
private BsaLabel() {}
|
||||
|
||||
BsaLabel(String label, DateTime creationTime) {
|
||||
this.label = label;
|
||||
|
|
|
@ -0,0 +1,153 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static google.registry.bsa.BsaStringUtils.DOMAIN_JOINER;
|
||||
import static google.registry.bsa.BsaStringUtils.DOMAIN_SPLITTER;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import google.registry.bsa.api.UnblockableDomain;
|
||||
import google.registry.bsa.persistence.BsaUnblockableDomain.BsaUnblockableDomainId;
|
||||
import google.registry.model.CreateAutoTimestamp;
|
||||
import google.registry.persistence.VKey;
|
||||
import java.io.Serializable;
|
||||
import javax.persistence.Column;
|
||||
import javax.persistence.Entity;
|
||||
import javax.persistence.EnumType;
|
||||
import javax.persistence.Enumerated;
|
||||
import javax.persistence.Id;
|
||||
import javax.persistence.IdClass;
|
||||
|
||||
/** A domain matching a BSA label but is in use (registered or reserved), so cannot be blocked. */
|
||||
@Entity
|
||||
@IdClass(BsaUnblockableDomainId.class)
|
||||
class BsaUnblockableDomain {
|
||||
@Id String label;
|
||||
@Id String tld;
|
||||
|
||||
@Column(nullable = false)
|
||||
@Enumerated(EnumType.STRING)
|
||||
Reason reason;
|
||||
|
||||
/**
|
||||
* Creation time of this record, which is the most recent time when the domain was detected to be
|
||||
* in use wrt BSA. It may be during the processing of a download, or during some other job that
|
||||
* refreshes the state.
|
||||
*
|
||||
* <p>This field is for information only.
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
@Column(nullable = false)
|
||||
CreateAutoTimestamp createTime = CreateAutoTimestamp.create(null);
|
||||
|
||||
// For Hibernate
|
||||
BsaUnblockableDomain() {}
|
||||
|
||||
BsaUnblockableDomain(String label, String tld, Reason reason) {
|
||||
this.label = label;
|
||||
this.tld = tld;
|
||||
this.reason = reason;
|
||||
}
|
||||
|
||||
String domainName() {
|
||||
return DOMAIN_JOINER.join(label, tld);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the equivalent {@link UnblockableDomain} instance, for use by communication with the
|
||||
* BSA API.
|
||||
*/
|
||||
UnblockableDomain toUnblockableDomain() {
|
||||
return UnblockableDomain.of(label, tld, UnblockableDomain.Reason.valueOf(reason.name()));
|
||||
}
|
||||
|
||||
VKey<BsaUnblockableDomain> toVkey() {
|
||||
return vKey(this.label, this.tld);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof BsaUnblockableDomain)) {
|
||||
return false;
|
||||
}
|
||||
BsaUnblockableDomain that = (BsaUnblockableDomain) o;
|
||||
return Objects.equal(label, that.label)
|
||||
&& Objects.equal(tld, that.tld)
|
||||
&& reason == that.reason
|
||||
&& Objects.equal(createTime, that.createTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(label, tld, reason, createTime);
|
||||
}
|
||||
|
||||
static BsaUnblockableDomain of(String domainName, Reason reason) {
|
||||
ImmutableList<String> parts = ImmutableList.copyOf(DOMAIN_SPLITTER.splitToList(domainName));
|
||||
verify(parts.size() == 2, "Invalid domain name: %s", domainName);
|
||||
return new BsaUnblockableDomain(parts.get(0), parts.get(1), reason);
|
||||
}
|
||||
|
||||
static VKey<BsaUnblockableDomain> vKey(String domainName) {
|
||||
ImmutableList<String> parts = ImmutableList.copyOf(DOMAIN_SPLITTER.splitToList(domainName));
|
||||
verify(parts.size() == 2, "Invalid domain name: %s", domainName);
|
||||
return vKey(parts.get(0), parts.get(1));
|
||||
}
|
||||
|
||||
static VKey<BsaUnblockableDomain> vKey(String label, String tld) {
|
||||
return VKey.create(BsaUnblockableDomain.class, new BsaUnblockableDomainId(label, tld));
|
||||
}
|
||||
|
||||
enum Reason {
|
||||
REGISTERED,
|
||||
RESERVED;
|
||||
}
|
||||
|
||||
static class BsaUnblockableDomainId implements Serializable {
|
||||
|
||||
private String label;
|
||||
private String tld;
|
||||
|
||||
@SuppressWarnings("unused") // For Hibernate
|
||||
BsaUnblockableDomainId() {}
|
||||
|
||||
BsaUnblockableDomainId(String label, String tld) {
|
||||
this.label = label;
|
||||
this.tld = tld;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof BsaUnblockableDomainId)) {
|
||||
return false;
|
||||
}
|
||||
BsaUnblockableDomainId that = (BsaUnblockableDomainId) o;
|
||||
return Objects.equal(label, that.label) && Objects.equal(tld, that.tld);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(label, tld);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,258 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static com.google.common.collect.ImmutableMap.toImmutableMap;
|
||||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||
import static google.registry.bsa.ReservedDomainsUtils.getAllReservedNames;
|
||||
import static google.registry.bsa.ReservedDomainsUtils.isReservedDomain;
|
||||
import static google.registry.bsa.persistence.Queries.queryLivesDomains;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static java.util.stream.Collectors.groupingBy;
|
||||
|
||||
import com.google.common.collect.ImmutableCollection;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.collect.Sets.SetView;
|
||||
import com.google.common.collect.Streams;
|
||||
import google.registry.bsa.BsaStringUtils;
|
||||
import google.registry.bsa.api.UnblockableDomain;
|
||||
import google.registry.bsa.api.UnblockableDomain.Reason;
|
||||
import google.registry.bsa.api.UnblockableDomainChange;
|
||||
import google.registry.model.ForeignKeyUtils;
|
||||
import google.registry.model.domain.Domain;
|
||||
import google.registry.util.BatchedStreams;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/**
|
||||
* Rechecks {@link BsaUnblockableDomain the registered/reserved domain names} in the database for
|
||||
* changes.
|
||||
*
|
||||
* <p>A registered/reserved domain name may change status in the following cases:
|
||||
*
|
||||
* <ul>
|
||||
* <li>A domain whose reason for being unblockable is `REGISTERED` will become blockable when the
|
||||
* domain is deregistered.
|
||||
* <li>A domain whose reason for being unblockable is `REGISTERED` will have its reason changed to
|
||||
* `RESERVED` if the domain is also on the reserved list.
|
||||
* <li>A domain whose reason for being unblockable is `RESERVED` will become blockable when the
|
||||
* domain is removed from the reserve list.
|
||||
* <li>A domain whose reason for being unblockable is `RESERVED` will have its reason changed to
|
||||
* `REGISTERED` if the domain is also on the reserved list.
|
||||
* <li>A blockable domain becomes unblockable when it is added to the reserve list.
|
||||
* <li>A blockable domain becomes unblockable when it is registered (with admin override).
|
||||
* </ul>
|
||||
*
|
||||
* <p>As a reminder, invalid domain names are not stored in the database. They change status only
|
||||
* when IDNs change in the TLDs, which rarely happens, and will be handled by dedicated procedures.
|
||||
*
|
||||
* <p>Domain blockability changes must be reported to BSA as follows:
|
||||
*
|
||||
* <ul>
|
||||
* <li>A blockable domain becoming unblockable: an addition
|
||||
* <li>An unblockable domain becoming blockable: a removal
|
||||
* <li>An unblockable domain with reason change: a removal followed by an insertion.
|
||||
* </ul>
|
||||
*
|
||||
* <p>Since BSA has separate endpoints for receiving blockability changes, removals must be sent
|
||||
* before additions.
|
||||
*/
|
||||
public final class DomainsRefresher {
|
||||
|
||||
private final DateTime prevRefreshStartTime;
|
||||
private final int transactionBatchSize;
|
||||
private final DateTime now;
|
||||
|
||||
public DomainsRefresher(
|
||||
DateTime prevRefreshStartTime,
|
||||
DateTime now,
|
||||
Duration domainTxnMaxDuration,
|
||||
int transactionBatchSize) {
|
||||
this.prevRefreshStartTime = prevRefreshStartTime.minus(domainTxnMaxDuration);
|
||||
this.now = now;
|
||||
this.transactionBatchSize = transactionBatchSize;
|
||||
}
|
||||
|
||||
public ImmutableList<UnblockableDomainChange> checkForBlockabilityChanges() {
|
||||
ImmutableList<UnblockableDomainChange> downgrades = refreshStaleUnblockables();
|
||||
ImmutableList<UnblockableDomainChange> upgrades = getNewUnblockables();
|
||||
|
||||
ImmutableSet<String> upgradedDomains =
|
||||
upgrades.stream().map(UnblockableDomainChange::domainName).collect(toImmutableSet());
|
||||
ImmutableList<UnblockableDomainChange> trueDowngrades =
|
||||
downgrades.stream()
|
||||
.filter(c -> !upgradedDomains.contains(c.domainName()))
|
||||
.collect(toImmutableList());
|
||||
return new ImmutableList.Builder<UnblockableDomainChange>()
|
||||
.addAll(upgrades)
|
||||
.addAll(trueDowngrades)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all changes to unblockable domains that have been reported to BSA. Please see {@link
|
||||
* UnblockableDomainChange} for types of possible changes. Note that invalid domain names are not
|
||||
* covered by this class and will be handled separately.
|
||||
*
|
||||
* <p>The number of changes are expected to be small for now. It is limited by the number of
|
||||
* domain deregistrations and the number of names added or removed from the reserved lists since
|
||||
* the previous refresh.
|
||||
*/
|
||||
public ImmutableList<UnblockableDomainChange> refreshStaleUnblockables() {
|
||||
ImmutableList.Builder<UnblockableDomainChange> changes = new ImmutableList.Builder<>();
|
||||
ImmutableList<BsaUnblockableDomain> batch;
|
||||
Optional<BsaUnblockableDomain> lastRead = Optional.empty();
|
||||
do {
|
||||
batch = Queries.batchReadUnblockables(lastRead, transactionBatchSize);
|
||||
if (!batch.isEmpty()) {
|
||||
lastRead = Optional.of(batch.get(batch.size() - 1));
|
||||
changes.addAll(recheckStaleDomainsBatch(batch));
|
||||
}
|
||||
} while (batch.size() == transactionBatchSize);
|
||||
return changes.build();
|
||||
}
|
||||
|
||||
ImmutableSet<UnblockableDomainChange> recheckStaleDomainsBatch(
|
||||
ImmutableList<BsaUnblockableDomain> domains) {
|
||||
ImmutableMap<String, BsaUnblockableDomain> nameToEntity =
|
||||
domains.stream().collect(toImmutableMap(BsaUnblockableDomain::domainName, d -> d));
|
||||
|
||||
ImmutableSet<String> prevRegistered =
|
||||
domains.stream()
|
||||
.filter(d -> d.reason.equals(BsaUnblockableDomain.Reason.REGISTERED))
|
||||
.map(BsaUnblockableDomain::domainName)
|
||||
.collect(toImmutableSet());
|
||||
ImmutableSet<String> currRegistered =
|
||||
ImmutableSet.copyOf(
|
||||
ForeignKeyUtils.load(Domain.class, nameToEntity.keySet(), now).keySet());
|
||||
SetView<String> noLongerRegistered = Sets.difference(prevRegistered, currRegistered);
|
||||
SetView<String> newlyRegistered = Sets.difference(currRegistered, prevRegistered);
|
||||
|
||||
ImmutableSet<String> prevReserved =
|
||||
domains.stream()
|
||||
.filter(d -> d.reason.equals(BsaUnblockableDomain.Reason.RESERVED))
|
||||
.map(BsaUnblockableDomain::domainName)
|
||||
.collect(toImmutableSet());
|
||||
ImmutableSet<String> currReserved =
|
||||
nameToEntity.keySet().stream()
|
||||
.filter(domain -> isReservedDomain(domain, now))
|
||||
.collect(toImmutableSet());
|
||||
SetView<String> noLongerReserved = Sets.difference(prevReserved, currReserved);
|
||||
|
||||
ImmutableSet.Builder<UnblockableDomainChange> changes = new ImmutableSet.Builder<>();
|
||||
// Newly registered: reserved -> registered
|
||||
for (String domainName : newlyRegistered) {
|
||||
BsaUnblockableDomain domain = nameToEntity.get(domainName);
|
||||
UnblockableDomain unblockable =
|
||||
UnblockableDomain.of(domain.label, domain.tld, Reason.valueOf(domain.reason.name()));
|
||||
changes.add(UnblockableDomainChange.ofChanged(unblockable, Reason.REGISTERED));
|
||||
}
|
||||
// No longer registered: registered -> reserved/NONE
|
||||
for (String domainName : noLongerRegistered) {
|
||||
BsaUnblockableDomain domain = nameToEntity.get(domainName);
|
||||
UnblockableDomain unblockable =
|
||||
UnblockableDomain.of(domain.label, domain.tld, Reason.valueOf(domain.reason.name()));
|
||||
changes.add(
|
||||
currReserved.contains(domainName)
|
||||
? UnblockableDomainChange.ofChanged(unblockable, Reason.RESERVED)
|
||||
: UnblockableDomainChange.ofDeleted(unblockable));
|
||||
}
|
||||
// No longer reserved: reserved -> registered/None (the former duplicates with newly-registered)
|
||||
for (String domainName : noLongerReserved) {
|
||||
BsaUnblockableDomain domain = nameToEntity.get(domainName);
|
||||
UnblockableDomain unblockable =
|
||||
UnblockableDomain.of(domain.label, domain.tld, Reason.valueOf(domain.reason.name()));
|
||||
if (!currRegistered.contains(domainName)) {
|
||||
changes.add(UnblockableDomainChange.ofDeleted(unblockable));
|
||||
}
|
||||
}
|
||||
return changes.build();
|
||||
}
|
||||
|
||||
public ImmutableList<UnblockableDomainChange> getNewUnblockables() {
|
||||
ImmutableSet<String> newCreated = getNewlyCreatedUnblockables(prevRefreshStartTime, now);
|
||||
ImmutableSet<String> newReserved = getNewlyReservedUnblockables(now, transactionBatchSize);
|
||||
SetView<String> reservedNotCreated = Sets.difference(newReserved, newCreated);
|
||||
return Streams.concat(
|
||||
newCreated.stream()
|
||||
.map(name -> UnblockableDomain.of(name, Reason.REGISTERED))
|
||||
.map(UnblockableDomainChange::ofNew),
|
||||
reservedNotCreated.stream()
|
||||
.map(name -> UnblockableDomain.of(name, Reason.RESERVED))
|
||||
.map(UnblockableDomainChange::ofNew))
|
||||
.collect(toImmutableList());
|
||||
}
|
||||
|
||||
static ImmutableSet<String> getNewlyCreatedUnblockables(
|
||||
DateTime prevRefreshStartTime, DateTime now) {
|
||||
ImmutableSet<String> liveDomains = queryLivesDomains(prevRefreshStartTime, now);
|
||||
return getUnblockedDomainNames(liveDomains);
|
||||
}
|
||||
|
||||
static ImmutableSet<String> getNewlyReservedUnblockables(DateTime now, int batchSize) {
|
||||
Stream<String> allReserved = getAllReservedNames(now);
|
||||
return BatchedStreams.toBatches(allReserved, batchSize)
|
||||
.map(DomainsRefresher::getUnblockedDomainNames)
|
||||
.flatMap(ImmutableSet::stream)
|
||||
.collect(toImmutableSet());
|
||||
}
|
||||
|
||||
static ImmutableSet<String> getUnblockedDomainNames(ImmutableCollection<String> domainNames) {
|
||||
Map<String, List<String>> labelToNames =
|
||||
domainNames.stream().collect(groupingBy(BsaStringUtils::getLabelInDomain));
|
||||
ImmutableSet<String> bsaLabels =
|
||||
Queries.queryBsaLabelByLabels(ImmutableSet.copyOf(labelToNames.keySet()))
|
||||
.map(BsaLabel::getLabel)
|
||||
.collect(toImmutableSet());
|
||||
return labelToNames.entrySet().stream()
|
||||
.filter(entry -> !bsaLabels.contains(entry.getKey()))
|
||||
.map(Entry::getValue)
|
||||
.flatMap(List::stream)
|
||||
.collect(toImmutableSet());
|
||||
}
|
||||
|
||||
public void applyUnblockableChanges(ImmutableList<UnblockableDomainChange> changes) {
|
||||
ImmutableMap<String, ImmutableSet<UnblockableDomainChange>> changesByType =
|
||||
ImmutableMap.copyOf(
|
||||
changes.stream()
|
||||
.collect(
|
||||
groupingBy(
|
||||
change -> change.isDelete() ? "remove" : "change", toImmutableSet())));
|
||||
tm().transact(
|
||||
() -> {
|
||||
if (changesByType.containsKey("remove")) {
|
||||
tm().delete(
|
||||
changesByType.get("remove").stream()
|
||||
.map(c -> BsaUnblockableDomain.vKey(c.domainName()))
|
||||
.collect(toImmutableSet()));
|
||||
}
|
||||
if (changesByType.containsKey("change")) {
|
||||
tm().putAll(
|
||||
changesByType.get("change").stream()
|
||||
.map(UnblockableDomainChange::newValue)
|
||||
.collect(toImmutableSet()));
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
|
@ -14,11 +14,19 @@
|
|||
|
||||
package google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static google.registry.bsa.DownloadStage.CHECKSUMS_DO_NOT_MATCH;
|
||||
import static google.registry.bsa.DownloadStage.MAKE_ORDER_AND_LABEL_DIFF;
|
||||
import static google.registry.bsa.DownloadStage.NOP;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import google.registry.bsa.BlockList;
|
||||
import google.registry.bsa.BlockListType;
|
||||
import google.registry.bsa.DownloadStage;
|
||||
import java.util.Optional;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/** Information needed when handling a download from BSA. */
|
||||
@AutoValue
|
||||
|
@ -26,6 +34,8 @@ public abstract class DownloadSchedule {
|
|||
|
||||
abstract long jobId();
|
||||
|
||||
abstract DateTime jobCreationTime();
|
||||
|
||||
public abstract String jobName();
|
||||
|
||||
public abstract DownloadStage stage();
|
||||
|
@ -37,11 +47,57 @@ public abstract class DownloadSchedule {
|
|||
* Returns true if download should be processed even if the checksums show that it has not changed
|
||||
* from the previous one.
|
||||
*/
|
||||
abstract boolean alwaysDownload();
|
||||
public abstract boolean alwaysDownload();
|
||||
|
||||
/** Updates the current job to the new stage. */
|
||||
public void updateJobStage(DownloadStage stage) {
|
||||
tm().transact(
|
||||
() -> {
|
||||
BsaDownload bsaDownload = tm().loadByKey(BsaDownload.vKey(jobId()));
|
||||
verify(
|
||||
stage.compareTo(bsaDownload.getStage()) > 0,
|
||||
"Invalid new stage [%s]. Must move forward from [%s]",
|
||||
bsaDownload.getStage(),
|
||||
stage);
|
||||
bsaDownload.setStage(stage);
|
||||
tm().put(bsaDownload);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the current job to the new stage and sets the checksums of the downloaded files.
|
||||
*
|
||||
* <p>This method may only be invoked during the {@code DOWNLOAD} stage, and the target stage must
|
||||
* be one of {@code MAKE_DIFF}, {@code CHECK_FOR_STALE_UNBLOCKABLES}, {@code NOP}, or {@code
|
||||
* CHECKSUMS_NOT_MATCH}.
|
||||
*/
|
||||
public DownloadSchedule updateJobStage(
|
||||
DownloadStage stage, ImmutableMap<BlockListType, String> checksums) {
|
||||
checkArgument(
|
||||
stage.equals(MAKE_ORDER_AND_LABEL_DIFF)
|
||||
|| stage.equals(NOP)
|
||||
|| stage.equals(CHECKSUMS_DO_NOT_MATCH),
|
||||
"Invalid stage [%s]",
|
||||
stage);
|
||||
return tm().transact(
|
||||
() -> {
|
||||
BsaDownload bsaDownload = tm().loadByKey(BsaDownload.vKey(jobId()));
|
||||
verify(
|
||||
bsaDownload.getStage().equals(DownloadStage.DOWNLOAD_BLOCK_LISTS),
|
||||
"Invalid invocation. May only invoke during the DOWNLOAD stage.",
|
||||
bsaDownload.getStage(),
|
||||
stage);
|
||||
bsaDownload.setStage(stage);
|
||||
bsaDownload.setChecksums(checksums);
|
||||
tm().put(bsaDownload);
|
||||
return of(bsaDownload);
|
||||
});
|
||||
}
|
||||
|
||||
static DownloadSchedule of(BsaDownload currentJob) {
|
||||
return new AutoValue_DownloadSchedule(
|
||||
currentJob.getJobId(),
|
||||
currentJob.getCreationTime(),
|
||||
currentJob.getJobName(),
|
||||
currentJob.getStage(),
|
||||
Optional.empty(),
|
||||
|
@ -52,6 +108,7 @@ public abstract class DownloadSchedule {
|
|||
BsaDownload currentJob, CompletedJob latestCompleted, boolean alwaysDownload) {
|
||||
return new AutoValue_DownloadSchedule(
|
||||
currentJob.getJobId(),
|
||||
currentJob.getCreationTime(),
|
||||
currentJob.getJobName(),
|
||||
currentJob.getStage(),
|
||||
Optional.of(latestCompleted),
|
||||
|
@ -63,7 +120,7 @@ public abstract class DownloadSchedule {
|
|||
public abstract static class CompletedJob {
|
||||
public abstract String jobName();
|
||||
|
||||
public abstract ImmutableMap<BlockList, String> checksums();
|
||||
public abstract ImmutableMap<BlockListType, String> checksums();
|
||||
|
||||
static CompletedJob of(BsaDownload completedJob) {
|
||||
return new AutoValue_DownloadSchedule_CompletedJob(
|
||||
|
|
|
@ -15,18 +15,22 @@
|
|||
package google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static google.registry.bsa.DownloadStage.CHECKSUMS_NOT_MATCH;
|
||||
import static google.registry.bsa.DownloadStage.CHECKSUMS_DO_NOT_MATCH;
|
||||
import static google.registry.bsa.DownloadStage.DONE;
|
||||
import static google.registry.bsa.DownloadStage.NOP;
|
||||
import static google.registry.bsa.persistence.RefreshScheduler.fetchMostRecentRefresh;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static org.joda.time.Duration.standardSeconds;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import google.registry.bsa.persistence.DownloadSchedule.CompletedJob;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.util.Clock;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import javax.inject.Inject;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/**
|
||||
|
@ -61,7 +65,10 @@ public final class DownloadScheduler {
|
|||
private final Clock clock;
|
||||
|
||||
@Inject
|
||||
DownloadScheduler(Duration downloadInterval, Duration maxNopInterval, Clock clock) {
|
||||
DownloadScheduler(
|
||||
@Config("bsaDownloadInterval") Duration downloadInterval,
|
||||
@Config("bsaMaxNopInterval") Duration maxNopInterval,
|
||||
Clock clock) {
|
||||
this.downloadInterval = downloadInterval;
|
||||
this.maxNopInterval = maxNopInterval;
|
||||
this.clock = clock;
|
||||
|
@ -71,26 +78,33 @@ public final class DownloadScheduler {
|
|||
* Returns a {@link DownloadSchedule} instance that describes the work to be performed by an
|
||||
* invocation of the download action, if applicable; or {@link Optional#empty} when there is
|
||||
* nothing to do.
|
||||
*
|
||||
* <p>For an interrupted job, work will resume from the {@link DownloadSchedule#stage}.
|
||||
*/
|
||||
public Optional<DownloadSchedule> schedule() {
|
||||
return tm().transact(
|
||||
() -> {
|
||||
ImmutableList<BsaDownload> recentJobs = loadRecentProcessedJobs();
|
||||
if (recentJobs.isEmpty()) {
|
||||
// No jobs initiated ever.
|
||||
ImmutableList<BsaDownload> recentDownloads = fetchTwoMostRecentDownloads();
|
||||
Optional<BsaDomainRefresh> mostRecentRefresh = fetchMostRecentRefresh();
|
||||
if (mostRecentRefresh.isPresent() && !mostRecentRefresh.get().isDone()) {
|
||||
// Ongoing refresh. Wait it out.
|
||||
return Optional.empty();
|
||||
}
|
||||
if (recentDownloads.isEmpty()) {
|
||||
// No downloads initiated ever.
|
||||
return Optional.of(scheduleNewJob(Optional.empty()));
|
||||
}
|
||||
BsaDownload mostRecent = recentJobs.get(0);
|
||||
BsaDownload mostRecent = recentDownloads.get(0);
|
||||
if (mostRecent.getStage().equals(DONE)) {
|
||||
return isTimeAgain(mostRecent, downloadInterval)
|
||||
? Optional.of(scheduleNewJob(Optional.of(mostRecent)))
|
||||
: Optional.empty();
|
||||
} else if (recentJobs.size() == 1) {
|
||||
} else if (recentDownloads.size() == 1) {
|
||||
// First job ever, still in progress
|
||||
return Optional.of(DownloadSchedule.of(recentJobs.get(0)));
|
||||
return Optional.of(DownloadSchedule.of(recentDownloads.get(0)));
|
||||
} else {
|
||||
// Job in progress, with completed previous jobs.
|
||||
BsaDownload prev = recentJobs.get(1);
|
||||
BsaDownload prev = recentDownloads.get(1);
|
||||
verify(prev.getStage().equals(DONE), "Unexpectedly found two ongoing jobs.");
|
||||
return Optional.of(
|
||||
DownloadSchedule.of(
|
||||
|
@ -101,6 +115,16 @@ public final class DownloadScheduler {
|
|||
});
|
||||
}
|
||||
|
||||
Optional<DateTime> latestCompletedJobTime() {
|
||||
return tm().transact(
|
||||
() -> {
|
||||
return fetchTwoMostRecentDownloads().stream()
|
||||
.filter(job -> Objects.equals(job.getStage(), DONE))
|
||||
.map(BsaDownload::getCreationTime)
|
||||
.findFirst();
|
||||
});
|
||||
}
|
||||
|
||||
private boolean isTimeAgain(BsaDownload mostRecent, Duration interval) {
|
||||
return mostRecent.getCreationTime().plus(interval).minus(CRON_JITTER).isBefore(clock.nowUtc());
|
||||
}
|
||||
|
@ -118,14 +142,25 @@ public final class DownloadScheduler {
|
|||
.orElseGet(() -> DownloadSchedule.of(job));
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches up to two most recent downloads, ordered by time in descending order. The first one may
|
||||
* be ongoing, and the second one (if exists) must be completed.
|
||||
*
|
||||
* <p>Jobs that do not download the data are ignored.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
ImmutableList<BsaDownload> loadRecentProcessedJobs() {
|
||||
static ImmutableList<BsaDownload> fetchTwoMostRecentDownloads() {
|
||||
return ImmutableList.copyOf(
|
||||
tm().getEntityManager()
|
||||
.createQuery(
|
||||
"FROM BsaDownload WHERE stage NOT IN :nop_stages ORDER BY creationTime DESC")
|
||||
.setParameter("nop_stages", ImmutableList.of(CHECKSUMS_NOT_MATCH, NOP))
|
||||
"FROM BsaDownload WHERE stage NOT IN :nop_stages ORDER BY creationTime DESC",
|
||||
BsaDownload.class)
|
||||
.setParameter("nop_stages", ImmutableList.of(CHECKSUMS_DO_NOT_MATCH, NOP))
|
||||
.setMaxResults(2)
|
||||
.getResultList());
|
||||
}
|
||||
|
||||
static Optional<BsaDownload> fetchMostRecentDownload() {
|
||||
return fetchTwoMostRecentDownloads().stream().findFirst();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,186 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||
import static com.google.common.collect.Sets.difference;
|
||||
import static google.registry.bsa.ReservedDomainsUtils.isReservedDomain;
|
||||
import static google.registry.persistence.PersistenceModule.TransactionIsolationLevel.TRANSACTION_REPEATABLE_READ;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static java.util.stream.Collectors.groupingBy;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.flogger.FluentLogger;
|
||||
import com.google.common.flogger.LazyArgs;
|
||||
import google.registry.bsa.IdnChecker;
|
||||
import google.registry.bsa.api.BlockLabel;
|
||||
import google.registry.bsa.api.BlockLabel.LabelType;
|
||||
import google.registry.bsa.api.UnblockableDomain;
|
||||
import google.registry.bsa.api.UnblockableDomain.Reason;
|
||||
import google.registry.model.ForeignKeyUtils;
|
||||
import google.registry.model.domain.Domain;
|
||||
import google.registry.model.tld.Tld;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/** Applies the BSA label diffs from the latest BSA download. */
|
||||
public final class LabelDiffUpdates {
|
||||
|
||||
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
|
||||
|
||||
private static final Joiner DOMAIN_JOINER = Joiner.on('.');
|
||||
|
||||
private LabelDiffUpdates() {}
|
||||
|
||||
/**
|
||||
* Applies the label diffs to the database and collects matching domains that are in use
|
||||
* (registered or reserved) for reporting.
|
||||
*
|
||||
* @return A collection of domains in use
|
||||
*/
|
||||
public static ImmutableList<UnblockableDomain> applyLabelDiff(
|
||||
ImmutableList<BlockLabel> labels,
|
||||
IdnChecker idnChecker,
|
||||
DownloadSchedule schedule,
|
||||
DateTime now) {
|
||||
ImmutableList.Builder<UnblockableDomain> nonBlockedDomains = new ImmutableList.Builder<>();
|
||||
ImmutableMap<LabelType, ImmutableList<BlockLabel>> labelsByType =
|
||||
ImmutableMap.copyOf(
|
||||
labels.stream().collect(groupingBy(BlockLabel::labelType, toImmutableList())));
|
||||
|
||||
tm().transact(
|
||||
() -> {
|
||||
for (Map.Entry<LabelType, ImmutableList<BlockLabel>> entry :
|
||||
labelsByType.entrySet()) {
|
||||
switch (entry.getKey()) {
|
||||
case CREATE:
|
||||
// With current Cloud SQL, label upsert throughput is about 200/second. If
|
||||
// better performance is needed, consider bulk insert in native SQL.
|
||||
tm().putAll(
|
||||
entry.getValue().stream()
|
||||
.filter(label -> isValidInAtLeastOneTld(label, idnChecker))
|
||||
.map(
|
||||
label ->
|
||||
new BsaLabel(label.label(), schedule.jobCreationTime()))
|
||||
.collect(toImmutableList()));
|
||||
// May not find all unblockables due to race condition: DomainCreateFlow uses
|
||||
// cached BsaLabels. Eventually will be consistent.
|
||||
nonBlockedDomains.addAll(
|
||||
tallyUnblockableDomainsForNewLabels(entry.getValue(), idnChecker, now));
|
||||
break;
|
||||
case DELETE:
|
||||
ImmutableSet<String> deletedLabels =
|
||||
entry.getValue().stream()
|
||||
.filter(label -> isValidInAtLeastOneTld(label, idnChecker))
|
||||
.map(BlockLabel::label)
|
||||
.collect(toImmutableSet());
|
||||
// Delete labels in DB. Also cascade-delete BsaUnblockableDomain.
|
||||
int nDeleted = Queries.deleteBsaLabelByLabels(deletedLabels);
|
||||
if (nDeleted != deletedLabels.size()) {
|
||||
logger.atSevere().log(
|
||||
"Only found %s entities among the %s labels: [%s]",
|
||||
nDeleted, deletedLabels.size(), deletedLabels);
|
||||
}
|
||||
break;
|
||||
case NEW_ORDER_ASSOCIATION:
|
||||
ImmutableSet<String> affectedLabels =
|
||||
entry.getValue().stream()
|
||||
.filter(label -> isValidInAtLeastOneTld(label, idnChecker))
|
||||
.map(BlockLabel::label)
|
||||
.collect(toImmutableSet());
|
||||
ImmutableSet<String> labelsInDb =
|
||||
Queries.queryBsaLabelByLabels(affectedLabels)
|
||||
.map(BsaLabel::getLabel)
|
||||
.collect(toImmutableSet());
|
||||
verify(
|
||||
labelsInDb.size() == affectedLabels.size(),
|
||||
"Missing labels in DB: %s",
|
||||
LazyArgs.lazy(() -> difference(affectedLabels, labelsInDb)));
|
||||
|
||||
// Reuse registered and reserved names that are already computed.
|
||||
Queries.queryBsaUnblockableDomainByLabels(affectedLabels)
|
||||
.map(BsaUnblockableDomain::toUnblockableDomain)
|
||||
.forEach(nonBlockedDomains::add);
|
||||
|
||||
for (BlockLabel label : entry.getValue()) {
|
||||
getInvalidTldsForLabel(label, idnChecker)
|
||||
.map(tld -> UnblockableDomain.of(label.label(), tld, Reason.INVALID))
|
||||
.forEach(nonBlockedDomains::add);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
},
|
||||
TRANSACTION_REPEATABLE_READ);
|
||||
logger.atInfo().log("Processed %s of labels.", labels.size());
|
||||
return nonBlockedDomains.build();
|
||||
}
|
||||
|
||||
static ImmutableList<UnblockableDomain> tallyUnblockableDomainsForNewLabels(
|
||||
ImmutableList<BlockLabel> labels, IdnChecker idnChecker, DateTime now) {
|
||||
ImmutableList.Builder<UnblockableDomain> nonBlockedDomains = new ImmutableList.Builder<>();
|
||||
|
||||
for (BlockLabel label : labels) {
|
||||
getInvalidTldsForLabel(label, idnChecker)
|
||||
.map(tld -> UnblockableDomain.of(label.label(), tld, Reason.INVALID))
|
||||
.forEach(nonBlockedDomains::add);
|
||||
}
|
||||
|
||||
ImmutableSet<String> validDomainNames =
|
||||
labels.stream()
|
||||
.map(label -> validDomainNamesForLabel(label, idnChecker))
|
||||
.flatMap(x -> x)
|
||||
.collect(toImmutableSet());
|
||||
ImmutableSet<String> registeredDomainNames =
|
||||
ImmutableSet.copyOf(ForeignKeyUtils.load(Domain.class, validDomainNames, now).keySet());
|
||||
for (String domain : registeredDomainNames) {
|
||||
nonBlockedDomains.add(UnblockableDomain.of(domain, Reason.REGISTERED));
|
||||
tm().put(BsaUnblockableDomain.of(domain, BsaUnblockableDomain.Reason.REGISTERED));
|
||||
}
|
||||
|
||||
ImmutableSet<String> reservedDomainNames =
|
||||
difference(validDomainNames, registeredDomainNames).stream()
|
||||
.filter(domain -> isReservedDomain(domain, now))
|
||||
.collect(toImmutableSet());
|
||||
for (String domain : reservedDomainNames) {
|
||||
nonBlockedDomains.add(UnblockableDomain.of(domain, Reason.RESERVED));
|
||||
tm().put(BsaUnblockableDomain.of(domain, BsaUnblockableDomain.Reason.RESERVED));
|
||||
}
|
||||
return nonBlockedDomains.build();
|
||||
}
|
||||
|
||||
static Stream<String> validDomainNamesForLabel(BlockLabel label, IdnChecker idnChecker) {
|
||||
return getValidTldsForLabel(label, idnChecker)
|
||||
.map(tld -> DOMAIN_JOINER.join(label.label(), tld));
|
||||
}
|
||||
|
||||
static Stream<String> getInvalidTldsForLabel(BlockLabel label, IdnChecker idnChecker) {
|
||||
return idnChecker.getForbiddingTlds(label.idnTables()).stream().map(Tld::getTldStr);
|
||||
}
|
||||
|
||||
static Stream<String> getValidTldsForLabel(BlockLabel label, IdnChecker idnChecker) {
|
||||
return idnChecker.getSupportingTlds(label.idnTables()).stream().map(Tld::getTldStr);
|
||||
}
|
||||
|
||||
static boolean isValidInAtLeastOneTld(BlockLabel label, IdnChecker idnChecker) {
|
||||
return getValidTldsForLabel(label, idnChecker).findAny().isPresent();
|
||||
}
|
||||
}
|
112
core/src/main/java/google/registry/bsa/persistence/Queries.java
Normal file
112
core/src/main/java/google/registry/bsa/persistence/Queries.java
Normal file
|
@ -0,0 +1,112 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static google.registry.bsa.BsaStringUtils.DOMAIN_SPLITTER;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
|
||||
import com.google.common.collect.ImmutableCollection;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.model.CreateAutoTimestamp;
|
||||
import google.registry.model.ForeignKeyUtils;
|
||||
import google.registry.model.domain.Domain;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/** Helpers for querying BSA JPA entities. */
|
||||
class Queries {
|
||||
|
||||
private Queries() {}
|
||||
|
||||
private static Object detach(Object obj) {
|
||||
tm().getEntityManager().detach(obj);
|
||||
return obj;
|
||||
}
|
||||
|
||||
static Stream<BsaUnblockableDomain> queryBsaUnblockableDomainByLabels(
|
||||
ImmutableCollection<String> labels) {
|
||||
return ((Stream<?>)
|
||||
tm().getEntityManager()
|
||||
.createQuery("FROM BsaUnblockableDomain WHERE label in (:labels)")
|
||||
.setParameter("labels", labels)
|
||||
.getResultStream())
|
||||
.map(Queries::detach)
|
||||
.map(BsaUnblockableDomain.class::cast);
|
||||
}
|
||||
|
||||
static Stream<BsaLabel> queryBsaLabelByLabels(ImmutableCollection<String> labels) {
|
||||
return ((Stream<?>)
|
||||
tm().getEntityManager()
|
||||
.createQuery("FROM BsaLabel where label in (:labels)")
|
||||
.setParameter("labels", labels)
|
||||
.getResultStream())
|
||||
.map(Queries::detach)
|
||||
.map(BsaLabel.class::cast);
|
||||
}
|
||||
|
||||
static int deleteBsaLabelByLabels(ImmutableCollection<String> labels) {
|
||||
return tm().getEntityManager()
|
||||
.createQuery("DELETE FROM BsaLabel where label IN (:deleted_labels)")
|
||||
.setParameter("deleted_labels", labels)
|
||||
.executeUpdate();
|
||||
}
|
||||
|
||||
static ImmutableList<BsaUnblockableDomain> batchReadUnblockables(
|
||||
Optional<BsaUnblockableDomain> lastRead, int batchSize) {
|
||||
return ImmutableList.copyOf(
|
||||
tm().getEntityManager()
|
||||
.createQuery(
|
||||
"FROM BsaUnblockableDomain d WHERE d.label > :label OR (d.label = :label AND d.tld"
|
||||
+ " > :tld) ORDER BY d.tld, d.label ")
|
||||
.setParameter("label", lastRead.map(d -> d.label).orElse(""))
|
||||
.setParameter("tld", lastRead.map(d -> d.tld).orElse(""))
|
||||
.setMaxResults(batchSize)
|
||||
.getResultList());
|
||||
}
|
||||
|
||||
static ImmutableSet<String> queryUnblockablesByNames(ImmutableSet<String> domains) {
|
||||
String labelTldParis =
|
||||
domains.stream()
|
||||
.map(
|
||||
domain -> {
|
||||
List<String> parts = DOMAIN_SPLITTER.splitToList(domain);
|
||||
verify(parts.size() == 2, "Invalid domain name %s", domain);
|
||||
return String.format("('%s','%s')", parts.get(0), parts.get(1));
|
||||
})
|
||||
.collect(Collectors.joining(","));
|
||||
String sql =
|
||||
String.format(
|
||||
"SELECT CONCAT(d.label, '.', d.tld) FROM \"BsaUnblockableDomain\" d "
|
||||
+ "WHERE (d.label, d.tld) IN (%s)",
|
||||
labelTldParis);
|
||||
return ImmutableSet.copyOf(tm().getEntityManager().createNativeQuery(sql).getResultList());
|
||||
}
|
||||
|
||||
static ImmutableSet<String> queryLivesDomains(DateTime minCreationTime, DateTime now) {
|
||||
ImmutableSet<String> candidates =
|
||||
ImmutableSet.copyOf(
|
||||
tm().getEntityManager()
|
||||
.createQuery(
|
||||
"SELECT domainName FROM Domain WHERE creationTime >= :time ", String.class)
|
||||
.setParameter("time", CreateAutoTimestamp.create(minCreationTime))
|
||||
.getResultList());
|
||||
return ImmutableSet.copyOf(ForeignKeyUtils.load(Domain.class, candidates, now).keySet());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
|
||||
import com.google.auto.value.AutoValue;
|
||||
import com.google.errorprone.annotations.CanIgnoreReturnValue;
|
||||
import google.registry.bsa.RefreshStage;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/** Information needed when handling a domain refresh. */
|
||||
@AutoValue
|
||||
public abstract class RefreshSchedule {
|
||||
|
||||
abstract long jobId();
|
||||
|
||||
abstract DateTime jobCreationTime();
|
||||
|
||||
public abstract String jobName();
|
||||
|
||||
public abstract RefreshStage stage();
|
||||
|
||||
/** The most recent job that ended in the {@code DONE} stage. */
|
||||
public abstract DateTime prevRefreshTime();
|
||||
|
||||
/** Updates the current job to the new stage. */
|
||||
@CanIgnoreReturnValue
|
||||
public RefreshSchedule updateJobStage(RefreshStage stage) {
|
||||
return tm().transact(
|
||||
() -> {
|
||||
BsaDomainRefresh bsaRefresh = tm().loadByKey(BsaDomainRefresh.vKey(jobId()));
|
||||
verify(
|
||||
stage.compareTo(bsaRefresh.getStage()) > 0,
|
||||
"Invalid new stage [%s]. Must move forward from [%s]",
|
||||
bsaRefresh.getStage(),
|
||||
stage);
|
||||
bsaRefresh.setStage(stage);
|
||||
tm().put(bsaRefresh);
|
||||
return of(bsaRefresh, prevRefreshTime());
|
||||
});
|
||||
}
|
||||
|
||||
static RefreshSchedule of(BsaDomainRefresh job, DateTime prevJobCreationTime) {
|
||||
return new AutoValue_RefreshSchedule(
|
||||
job.getJobId(),
|
||||
job.getCreationTime(),
|
||||
job.getJobName(),
|
||||
job.getStage(),
|
||||
prevJobCreationTime);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static google.registry.bsa.persistence.DownloadScheduler.fetchMostRecentDownload;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import java.util.Optional;
|
||||
import javax.inject.Inject;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/** Assigns work for each cron invocation of domain refresh job. */
|
||||
public class RefreshScheduler {
|
||||
|
||||
@Inject
|
||||
RefreshScheduler() {}
|
||||
|
||||
public Optional<RefreshSchedule> schedule() {
|
||||
return tm().transact(
|
||||
() -> {
|
||||
ImmutableList<BsaDomainRefresh> recentJobs = fetchMostRecentRefreshes();
|
||||
Optional<BsaDownload> mostRecentDownload = fetchMostRecentDownload();
|
||||
if (mostRecentDownload.isPresent() && !mostRecentDownload.get().isDone()) {
|
||||
// Ongoing download exists. Must wait it out.
|
||||
return Optional.empty();
|
||||
}
|
||||
if (recentJobs.size() > 1) {
|
||||
BsaDomainRefresh mostRecent = recentJobs.get(0);
|
||||
if (mostRecent.isDone()) {
|
||||
return Optional.of(scheduleNewJob(mostRecent.getCreationTime()));
|
||||
} else {
|
||||
return Optional.of(
|
||||
rescheduleOngoingJob(mostRecent, recentJobs.get(1).getCreationTime()));
|
||||
}
|
||||
}
|
||||
if (recentJobs.size() == 1 && recentJobs.get(0).isDone()) {
|
||||
return Optional.of(scheduleNewJob(recentJobs.get(0).getCreationTime()));
|
||||
}
|
||||
// No previously completed refreshes. Need start time of a completed download as
|
||||
// lower bound of refresh checks.
|
||||
if (!mostRecentDownload.isPresent()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
DateTime prevDownloadTime = mostRecentDownload.get().getCreationTime();
|
||||
if (recentJobs.isEmpty()) {
|
||||
return Optional.of(scheduleNewJob(prevDownloadTime));
|
||||
} else {
|
||||
return Optional.of(rescheduleOngoingJob(recentJobs.get(0), prevDownloadTime));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
RefreshSchedule scheduleNewJob(DateTime prevRefreshTime) {
|
||||
BsaDomainRefresh newJob = new BsaDomainRefresh();
|
||||
tm().insert(newJob);
|
||||
return RefreshSchedule.of(newJob, prevRefreshTime);
|
||||
}
|
||||
|
||||
RefreshSchedule rescheduleOngoingJob(BsaDomainRefresh ongoingJob, DateTime prevJobStartTime) {
|
||||
return RefreshSchedule.of(ongoingJob, prevJobStartTime);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static ImmutableList<BsaDomainRefresh> fetchMostRecentRefreshes() {
|
||||
return ImmutableList.copyOf(
|
||||
tm().getEntityManager()
|
||||
.createQuery("FROM BsaDomainRefresh ORDER BY creationTime DESC", BsaDomainRefresh.class)
|
||||
.setMaxResults(2)
|
||||
.getResultList());
|
||||
}
|
||||
|
||||
static Optional<BsaDomainRefresh> fetchMostRecentRefresh() {
|
||||
return fetchMostRecentRefreshes().stream().findFirst();
|
||||
}
|
||||
}
|
|
@ -1445,9 +1445,15 @@ public final class RegistryConfig {
|
|||
}
|
||||
|
||||
@Provides
|
||||
@Config("bsaLabelTxnBatchSize")
|
||||
public static int provideBsaLabelTxnBatchSize(RegistryConfigSettings config) {
|
||||
return config.bsa.bsaLabelTxnBatchSize;
|
||||
@Config("bsaTxnBatchSize")
|
||||
public static int provideBsaTxnBatchSize(RegistryConfigSettings config) {
|
||||
return config.bsa.bsaTxnBatchSize;
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Config("domainTxnMaxDuration")
|
||||
public static Duration provideDomainTxnMaxDuration(RegistryConfigSettings config) {
|
||||
return Duration.standardSeconds(config.bsa.domainTxnMaxDurationSeconds);
|
||||
}
|
||||
|
||||
@Provides
|
||||
|
|
|
@ -272,7 +272,8 @@ public class RegistryConfigSettings {
|
|||
public int bsaLockLeaseExpiryMinutes;
|
||||
public int bsaDownloadIntervalMinutes;
|
||||
public int bsaMaxNopIntervalHours;
|
||||
public int bsaLabelTxnBatchSize;
|
||||
public int bsaTxnBatchSize;
|
||||
public int domainTxnMaxDurationSeconds;
|
||||
public String authUrl;
|
||||
public int authTokenExpirySeconds;
|
||||
public Map<String, String> dataUrls;
|
||||
|
|
|
@ -615,6 +615,22 @@ bulkPricingPackageMonitoring:
|
|||
|
||||
# Configurations for integration with Brand Safety Alliance (BSA) API
|
||||
bsa:
|
||||
# Algorithm for calculating block list checksums
|
||||
bsaChecksumAlgorithm: SHA-256
|
||||
# The time allotted to every BSA cron job.
|
||||
bsaLockLeaseExpiryMinutes: 30
|
||||
# Desired time between successive downloads.
|
||||
bsaDownloadIntervalMinutes: 30
|
||||
# Max time period during which downloads can be skipped because checksums have
|
||||
# not changed from the previous one.
|
||||
bsaMaxNopIntervalHours: 24
|
||||
# A very lax upper bound of the time it takes to execute a transaction that
|
||||
# mutates a domain. Please See `BsaRefreshAction` for use case.
|
||||
domainTxnMaxDurationSeconds: 60
|
||||
# Number of entities (labels and unblockable domains) to process in a single
|
||||
# DB transaction.
|
||||
bsaTxnBatchSize: 1000
|
||||
|
||||
# Http endpoint for acquiring Auth tokens.
|
||||
authUrl: "https://"
|
||||
# Auth token expiry.
|
||||
|
|
|
@ -13,12 +13,18 @@
|
|||
<load-on-startup>1</load-on-startup>
|
||||
</servlet>
|
||||
|
||||
<!-- Test action -->
|
||||
<!-- Download action -->
|
||||
<servlet-mapping>
|
||||
<servlet-name>bsa-servlet</servlet-name>
|
||||
<url-pattern>/_dr/task/bsaDownload</url-pattern>
|
||||
</servlet-mapping>
|
||||
|
||||
<!-- Refresh action -->
|
||||
<servlet-mapping>
|
||||
<servlet-name>bsa-servlet</servlet-name>
|
||||
<url-pattern>/_dr/task/bsaRefresh</url-pattern>
|
||||
</servlet-mapping>
|
||||
|
||||
<!-- Security config -->
|
||||
<security-constraint>
|
||||
<web-resource-collection>
|
||||
|
|
|
@ -519,7 +519,7 @@ public class DomainFlowUtils {
|
|||
private static final ImmutableSet<ReservationType> RESERVED_TYPES =
|
||||
ImmutableSet.of(RESERVED_FOR_SPECIFIC_USE, RESERVED_FOR_ANCHOR_TENANT, FULLY_BLOCKED);
|
||||
|
||||
static boolean isReserved(InternetDomainName domainName, boolean isSunrise) {
|
||||
public static boolean isReserved(InternetDomainName domainName, boolean isSunrise) {
|
||||
ImmutableSet<ReservationType> types = getReservationTypes(domainName);
|
||||
return !Sets.intersection(types, RESERVED_TYPES).isEmpty()
|
||||
|| !(isSunrise || intersection(TYPES_ALLOWED_FOR_CREATE_ONLY_IN_SUNRISE, types).isEmpty());
|
||||
|
|
|
@ -22,6 +22,7 @@ import static com.google.common.base.Strings.emptyToNull;
|
|||
import static com.google.common.collect.ImmutableSet.toImmutableSet;
|
||||
import static com.google.common.collect.Maps.filterValues;
|
||||
import static google.registry.model.CacheUtils.memoizeWithShortExpiration;
|
||||
import static google.registry.model.tld.Tld.isEnrolledWithBsa;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.util.CollectionUtils.entriesToImmutableMap;
|
||||
import static google.registry.util.PreconditionsUtils.checkArgumentNotNull;
|
||||
|
@ -38,6 +39,7 @@ import java.util.Optional;
|
|||
import java.util.function.Supplier;
|
||||
import java.util.stream.Stream;
|
||||
import javax.persistence.EntityManager;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/** Utilities for finding and listing {@link Tld} entities. */
|
||||
public final class Tlds {
|
||||
|
@ -151,4 +153,9 @@ public final class Tlds {
|
|||
findTldForName(domainName).orElse(null),
|
||||
"Domain name is not under a recognized TLD: %s", domainName.toString());
|
||||
}
|
||||
|
||||
/** Returns true if at least one TLD is enrolled {@code now}. */
|
||||
public static boolean hasActiveBsaEnrollment(DateTime now) {
|
||||
return getTldEntitiesOfType(TldType.REAL).stream().anyMatch(tld -> isEnrolledWithBsa(tld, now));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,9 +19,14 @@ import dagger.Component;
|
|||
import dagger.Lazy;
|
||||
import google.registry.config.CredentialModule;
|
||||
import google.registry.config.RegistryConfig.ConfigModule;
|
||||
import google.registry.keyring.KeyringModule;
|
||||
import google.registry.keyring.secretmanager.SecretManagerKeyringModule;
|
||||
import google.registry.module.bsa.BsaRequestComponent.BsaRequestComponentModule;
|
||||
import google.registry.monitoring.whitebox.StackdriverModule;
|
||||
import google.registry.persistence.PersistenceModule;
|
||||
import google.registry.privileges.secretmanager.SecretManagerModule;
|
||||
import google.registry.request.Modules.GsonModule;
|
||||
import google.registry.request.Modules.UrlConnectionServiceModule;
|
||||
import google.registry.request.Modules.UserServiceModule;
|
||||
import google.registry.request.auth.AuthModule;
|
||||
import google.registry.util.UtilsModule;
|
||||
|
@ -31,13 +36,18 @@ import javax.inject.Singleton;
|
|||
@Component(
|
||||
modules = {
|
||||
AuthModule.class,
|
||||
UtilsModule.class,
|
||||
UserServiceModule.class,
|
||||
GsonModule.class,
|
||||
BsaRequestComponentModule.class,
|
||||
ConfigModule.class,
|
||||
StackdriverModule.class,
|
||||
CredentialModule.class,
|
||||
BsaRequestComponentModule.class
|
||||
GsonModule.class,
|
||||
PersistenceModule.class,
|
||||
KeyringModule.class,
|
||||
SecretManagerKeyringModule.class,
|
||||
SecretManagerModule.class,
|
||||
StackdriverModule.class,
|
||||
UrlConnectionServiceModule.class,
|
||||
UserServiceModule.class,
|
||||
UtilsModule.class
|
||||
})
|
||||
interface BsaComponent {
|
||||
BsaRequestHandler requestHandler();
|
||||
|
|
|
@ -16,7 +16,8 @@ package google.registry.module.bsa;
|
|||
|
||||
import dagger.Module;
|
||||
import dagger.Subcomponent;
|
||||
import google.registry.bsa.PlaceholderAction;
|
||||
import google.registry.bsa.BsaDownloadAction;
|
||||
import google.registry.bsa.BsaRefreshAction;
|
||||
import google.registry.request.RequestComponentBuilder;
|
||||
import google.registry.request.RequestModule;
|
||||
import google.registry.request.RequestScope;
|
||||
|
@ -28,7 +29,9 @@ import google.registry.request.RequestScope;
|
|||
})
|
||||
interface BsaRequestComponent {
|
||||
|
||||
PlaceholderAction bsaAction();
|
||||
BsaDownloadAction bsaDownloadAction();
|
||||
|
||||
BsaRefreshAction bsaRefreshAction();
|
||||
|
||||
@Subcomponent.Builder
|
||||
abstract class Builder implements RequestComponentBuilder<BsaRequestComponent> {
|
||||
|
|
|
@ -41,7 +41,7 @@
|
|||
<class>google.registry.bsa.persistence.BsaDomainRefresh</class>
|
||||
<class>google.registry.bsa.persistence.BsaDownload</class>
|
||||
<class>google.registry.bsa.persistence.BsaLabel</class>
|
||||
<class>google.registry.bsa.persistence.BsaDomainInUse</class>
|
||||
<class>google.registry.bsa.persistence.BsaUnblockableDomain</class>
|
||||
<class>google.registry.model.billing.BillingCancellation</class>
|
||||
<class>google.registry.model.billing.BillingEvent</class>
|
||||
<class>google.registry.model.billing.BillingRecurrence</class>
|
||||
|
|
177
core/src/test/java/google/registry/bsa/BlockListFetcherTest.java
Normal file
177
core/src/test/java/google/registry/bsa/BlockListFetcherTest.java
Normal file
|
@ -0,0 +1,177 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import google.registry.bsa.BlockListFetcher.LazyBlockList;
|
||||
import google.registry.bsa.api.BsaCredential;
|
||||
import google.registry.bsa.api.BsaException;
|
||||
import google.registry.request.UrlConnectionService;
|
||||
import google.registry.util.Retrier;
|
||||
import google.registry.util.SystemSleeper;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.security.GeneralSecurityException;
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
/** Unit tests for {@link BlockListFetcher}. */
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
class BlockListFetcherTest {
|
||||
@Mock HttpsURLConnection connection;
|
||||
@Mock UrlConnectionService connectionService;
|
||||
@Mock BsaCredential credential;
|
||||
|
||||
BlockListFetcher fetcher;
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
fetcher =
|
||||
new BlockListFetcher(
|
||||
connectionService,
|
||||
credential,
|
||||
ImmutableMap.of("BLOCK", "https://block", "BLOCK_PLUS", "https://blockplus"),
|
||||
new Retrier(new SystemSleeper(), 2));
|
||||
}
|
||||
|
||||
void setupMocks() throws Exception {
|
||||
when(connectionService.createConnection(any(URL.class))).thenReturn(connection);
|
||||
when(credential.getAuthToken()).thenReturn("authToken");
|
||||
}
|
||||
|
||||
@Test
|
||||
void tryFetch_bsaChecksumFetched() throws Exception {
|
||||
setupMocks();
|
||||
when(connection.getResponseCode()).thenReturn(SC_OK);
|
||||
when(connection.getInputStream())
|
||||
.thenReturn(new ByteArrayInputStream("bsa-checksum\ndata".getBytes(UTF_8)));
|
||||
LazyBlockList download = fetcher.tryFetch(BlockListType.BLOCK);
|
||||
assertThat(download.getName()).isEqualTo(BlockListType.BLOCK);
|
||||
assertThat(download.checksum()).isEqualTo("bsa-checksum");
|
||||
verify(connection, times(1)).setRequestMethod("GET");
|
||||
verify(connection, times(1)).setRequestProperty("Authorization", "Bearer authToken");
|
||||
}
|
||||
|
||||
@Test
|
||||
void tryFetch_ifStatusNotOK_throwRetriable() throws Exception {
|
||||
setupMocks();
|
||||
when(connection.getResponseCode()).thenReturn(201);
|
||||
assertThat(
|
||||
assertThrows(BsaException.class, () -> fetcher.tryFetch(BlockListType.BLOCK))
|
||||
.isRetriable())
|
||||
.isTrue();
|
||||
}
|
||||
|
||||
@Test
|
||||
void tryFetch_IOException_retriable() throws Exception {
|
||||
setupMocks();
|
||||
when(connection.getResponseCode()).thenThrow(new IOException());
|
||||
assertThat(
|
||||
assertThrows(BsaException.class, () -> fetcher.tryFetch(BlockListType.BLOCK))
|
||||
.isRetriable())
|
||||
.isTrue();
|
||||
}
|
||||
|
||||
@Test
|
||||
void tryFetch_SecurityException_notRetriable() throws Exception {
|
||||
when(connectionService.createConnection(any(URL.class)))
|
||||
.thenThrow(new GeneralSecurityException());
|
||||
assertThat(
|
||||
assertThrows(BsaException.class, () -> fetcher.tryFetch(BlockListType.BLOCK))
|
||||
.isRetriable())
|
||||
.isFalse();
|
||||
}
|
||||
|
||||
@Test
|
||||
void lazyBlock_blockListFetched() throws Exception {
|
||||
setupMocks();
|
||||
when(connection.getInputStream())
|
||||
.thenReturn(new ByteArrayInputStream("bsa-checksum\ndata".getBytes(UTF_8)));
|
||||
when(connection.getResponseCode()).thenReturn(SC_OK);
|
||||
try (LazyBlockList download = fetcher.tryFetch(BlockListType.BLOCK)) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
download.consumeAll(
|
||||
(buffer, length) -> {
|
||||
String snippet = new String(buffer, 0, length, UTF_8);
|
||||
sb.append(snippet);
|
||||
});
|
||||
assertThat(sb.toString()).isEqualTo("data");
|
||||
}
|
||||
verify(connection, times(1)).disconnect();
|
||||
}
|
||||
|
||||
@Test
|
||||
void lazyBlockPlus_success() throws Exception {
|
||||
setupMocks();
|
||||
when(connection.getInputStream())
|
||||
.thenReturn(new ByteArrayInputStream("checksum\ndata\n".getBytes(UTF_8)));
|
||||
when(connection.getResponseCode()).thenReturn(SC_OK);
|
||||
try (LazyBlockList lazyBlockList = fetcher.tryFetch(BlockListType.BLOCK_PLUS)) {
|
||||
assertThat(readBlockData(lazyBlockList)).isEqualTo("data\n");
|
||||
assertThat(lazyBlockList.checksum()).isEqualTo("checksum");
|
||||
}
|
||||
verify(connection, times(1)).disconnect();
|
||||
}
|
||||
|
||||
@Test
|
||||
void lazyBlockPlus_checksum_cr() throws Exception {
|
||||
setupMocks();
|
||||
when(connection.getInputStream())
|
||||
.thenReturn(new ByteArrayInputStream("checksum\rdata\n".getBytes(UTF_8)));
|
||||
when(connection.getResponseCode()).thenReturn(SC_OK);
|
||||
try (LazyBlockList lazyBlockList = fetcher.tryFetch(BlockListType.BLOCK_PLUS)) {
|
||||
assertThat(readBlockData(lazyBlockList)).isEqualTo("data\n");
|
||||
assertThat(lazyBlockList.checksum()).isEqualTo("checksum");
|
||||
}
|
||||
verify(connection, times(1)).disconnect();
|
||||
}
|
||||
|
||||
@Test
|
||||
void lazyBlockPlus_checksum_crnl() throws Exception {
|
||||
setupMocks();
|
||||
when(connection.getInputStream())
|
||||
.thenReturn(new ByteArrayInputStream("checksum\r\ndata\n".getBytes(UTF_8)));
|
||||
when(connection.getResponseCode()).thenReturn(SC_OK);
|
||||
try (LazyBlockList lazyBlockList = fetcher.tryFetch(BlockListType.BLOCK_PLUS)) {
|
||||
assertThat(readBlockData(lazyBlockList)).isEqualTo("data\n");
|
||||
assertThat(lazyBlockList.checksum()).isEqualTo("checksum");
|
||||
}
|
||||
verify(connection, times(1)).disconnect();
|
||||
}
|
||||
|
||||
private String readBlockData(LazyBlockList lazyBlockList) throws Exception {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
lazyBlockList.consumeAll(
|
||||
(buffer, length) -> {
|
||||
String snippet = new String(buffer, 0, length, UTF_8);
|
||||
sb.append(snippet);
|
||||
});
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
302
core/src/test/java/google/registry/bsa/BsaDiffCreatorTest.java
Normal file
302
core/src/test/java/google/registry/bsa/BsaDiffCreatorTest.java
Normal file
|
@ -0,0 +1,302 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.bsa.BsaDiffCreator.ORDER_ID_SENTINEL;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
import static org.mockito.ArgumentMatchers.anyString;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.bsa.BsaDiffCreator.BsaDiff;
|
||||
import google.registry.bsa.BsaDiffCreator.Canonicals;
|
||||
import google.registry.bsa.BsaDiffCreator.LabelOrderPair;
|
||||
import google.registry.bsa.BsaDiffCreator.Line;
|
||||
import google.registry.bsa.api.BlockLabel;
|
||||
import google.registry.bsa.api.BlockLabel.LabelType;
|
||||
import google.registry.bsa.api.BlockOrder;
|
||||
import google.registry.bsa.api.BlockOrder.OrderType;
|
||||
import google.registry.bsa.persistence.DownloadSchedule;
|
||||
import google.registry.bsa.persistence.DownloadSchedule.CompletedJob;
|
||||
import google.registry.tldconfig.idn.IdnTableEnum;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
/** Unit tests for {@link BsaDiffCreator}. */
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
class BsaDiffCreatorTest {
|
||||
|
||||
@Mock GcsClient gcsClient;
|
||||
|
||||
@Mock DownloadSchedule schedule;
|
||||
@Mock CompletedJob completedJob;
|
||||
@Mock IdnChecker idnChecker;
|
||||
|
||||
BsaDiffCreator diffCreator;
|
||||
|
||||
@Test
|
||||
void firstDiff() {
|
||||
when(idnChecker.getAllValidIdns(anyString())).thenReturn(ImmutableSet.of(IdnTableEnum.JA));
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList(anyString(), eq(BlockListType.BLOCK_PLUS)))
|
||||
.thenAnswer((ignore) -> Stream.of());
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.empty());
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels())
|
||||
.containsExactly(
|
||||
BlockLabel.of("test1", LabelType.CREATE, ImmutableSet.of("JA")),
|
||||
BlockLabel.of("test2", LabelType.CREATE, ImmutableSet.of("JA")),
|
||||
BlockLabel.of("test3", LabelType.CREATE, ImmutableSet.of("JA")));
|
||||
assertThat(diff.getOrders())
|
||||
.containsExactly(
|
||||
BlockOrder.of(1, OrderType.CREATE),
|
||||
BlockOrder.of(2, OrderType.CREATE),
|
||||
BlockOrder.of(3, OrderType.CREATE),
|
||||
BlockOrder.of(4, OrderType.CREATE));
|
||||
}
|
||||
|
||||
@Test
|
||||
void firstDiff_labelMultipleOccurrences() {
|
||||
when(idnChecker.getAllValidIdns(anyString())).thenReturn(ImmutableSet.of(IdnTableEnum.JA));
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK_PLUS))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,5"));
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.empty());
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels())
|
||||
.containsExactly(
|
||||
BlockLabel.of("test1", LabelType.CREATE, ImmutableSet.of("JA")),
|
||||
BlockLabel.of("test2", LabelType.CREATE, ImmutableSet.of("JA")),
|
||||
BlockLabel.of("test3", LabelType.CREATE, ImmutableSet.of("JA")));
|
||||
assertThat(diff.getOrders())
|
||||
.containsExactly(
|
||||
BlockOrder.of(1, OrderType.CREATE),
|
||||
BlockOrder.of(2, OrderType.CREATE),
|
||||
BlockOrder.of(3, OrderType.CREATE),
|
||||
BlockOrder.of(4, OrderType.CREATE),
|
||||
BlockOrder.of(5, OrderType.CREATE));
|
||||
}
|
||||
|
||||
@Test
|
||||
void unchanged() {
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList("second", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList(anyString(), eq(BlockListType.BLOCK_PLUS)))
|
||||
.thenAnswer((ignore) -> Stream.of());
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("second");
|
||||
when(completedJob.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.of(completedJob));
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels()).isEmpty();
|
||||
assertThat(diff.getOrders()).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void allRemoved() {
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList("second", BlockListType.BLOCK)).thenReturn(Stream.of());
|
||||
when(gcsClient.readBlockList(anyString(), eq(BlockListType.BLOCK_PLUS)))
|
||||
.thenAnswer((ignore) -> Stream.of());
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("second");
|
||||
when(completedJob.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.of(completedJob));
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels())
|
||||
.containsExactly(
|
||||
BlockLabel.of("test1", LabelType.DELETE, ImmutableSet.of()),
|
||||
BlockLabel.of("test2", LabelType.DELETE, ImmutableSet.of()),
|
||||
BlockLabel.of("test3", LabelType.DELETE, ImmutableSet.of()));
|
||||
assertThat(diff.getOrders())
|
||||
.containsExactly(
|
||||
BlockOrder.of(1, OrderType.DELETE),
|
||||
BlockOrder.of(2, OrderType.DELETE),
|
||||
BlockOrder.of(3, OrderType.DELETE),
|
||||
BlockOrder.of(4, OrderType.DELETE));
|
||||
}
|
||||
|
||||
@Test
|
||||
void existingLabelNewOrder() {
|
||||
when(idnChecker.getAllValidIdns(anyString())).thenReturn(ImmutableSet.of(IdnTableEnum.JA));
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList("second", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2;5", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList(anyString(), eq(BlockListType.BLOCK_PLUS)))
|
||||
.thenAnswer((ignore) -> Stream.of());
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("second");
|
||||
when(completedJob.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.of(completedJob));
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels())
|
||||
.containsExactly(
|
||||
BlockLabel.of("test1", LabelType.NEW_ORDER_ASSOCIATION, ImmutableSet.of("JA")));
|
||||
assertThat(diff.getOrders()).containsExactly(BlockOrder.of(5, OrderType.CREATE));
|
||||
}
|
||||
|
||||
@Test
|
||||
void newLabelNewOrder() {
|
||||
when(idnChecker.getAllValidIdns(anyString())).thenReturn(ImmutableSet.of(IdnTableEnum.JA));
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList("second", BlockListType.BLOCK))
|
||||
.thenReturn(
|
||||
Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4", "test4,5"));
|
||||
when(gcsClient.readBlockList(anyString(), eq(BlockListType.BLOCK_PLUS)))
|
||||
.thenAnswer((ignore) -> Stream.of());
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("second");
|
||||
when(completedJob.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.of(completedJob));
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels())
|
||||
.containsExactly(BlockLabel.of("test4", LabelType.CREATE, ImmutableSet.of("JA")));
|
||||
assertThat(diff.getOrders()).containsExactly(BlockOrder.of(5, OrderType.CREATE));
|
||||
}
|
||||
|
||||
@Test
|
||||
void removeOrderOnly() {
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList("second", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1"));
|
||||
when(gcsClient.readBlockList(anyString(), eq(BlockListType.BLOCK_PLUS)))
|
||||
.thenAnswer((ignore) -> Stream.of());
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("second");
|
||||
when(completedJob.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.of(completedJob));
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels()).isEmpty();
|
||||
assertThat(diff.getOrders()).containsExactly(BlockOrder.of(4, OrderType.DELETE));
|
||||
}
|
||||
|
||||
@Test
|
||||
void removeOrderOnly_multiLabelOrder() {
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList("second", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,2", "test2,3", "test3,4"));
|
||||
when(gcsClient.readBlockList(anyString(), eq(BlockListType.BLOCK_PLUS)))
|
||||
.thenAnswer((ignore) -> Stream.of());
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("second");
|
||||
when(completedJob.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.of(completedJob));
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels()).isEmpty();
|
||||
assertThat(diff.getOrders()).containsExactly(BlockOrder.of(1, OrderType.DELETE));
|
||||
}
|
||||
|
||||
@Test
|
||||
void removeLabelAndOrder() {
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList("second", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test3,1;4"));
|
||||
when(gcsClient.readBlockList(anyString(), eq(BlockListType.BLOCK_PLUS)))
|
||||
.thenAnswer((ignore) -> Stream.of());
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("second");
|
||||
when(completedJob.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.of(completedJob));
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels())
|
||||
.containsExactly(BlockLabel.of("test2", LabelType.DELETE, ImmutableSet.of()));
|
||||
assertThat(diff.getOrders()).containsExactly(BlockOrder.of(3, OrderType.DELETE));
|
||||
}
|
||||
|
||||
@Test
|
||||
void removeLabelAndOrder_multi() {
|
||||
when(gcsClient.readBlockList("first", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test1,1;2", "test2,3", "test3,1;4"));
|
||||
when(gcsClient.readBlockList("second", BlockListType.BLOCK))
|
||||
.thenReturn(Stream.of("domainLabel,orderIDs", "test2,3"));
|
||||
when(gcsClient.readBlockList(anyString(), eq(BlockListType.BLOCK_PLUS)))
|
||||
.thenAnswer((ignore) -> Stream.of());
|
||||
diffCreator = new BsaDiffCreator(gcsClient);
|
||||
when(schedule.jobName()).thenReturn("second");
|
||||
when(completedJob.jobName()).thenReturn("first");
|
||||
when(schedule.latestCompleted()).thenReturn(Optional.of(completedJob));
|
||||
BsaDiff diff = diffCreator.createDiff(schedule, idnChecker);
|
||||
assertThat(diff.getLabels())
|
||||
.containsExactly(
|
||||
BlockLabel.of("test1", LabelType.DELETE, ImmutableSet.of()),
|
||||
BlockLabel.of("test3", LabelType.DELETE, ImmutableSet.of()));
|
||||
assertThat(diff.getOrders())
|
||||
.containsExactly(
|
||||
BlockOrder.of(1, OrderType.DELETE),
|
||||
BlockOrder.of(2, OrderType.DELETE),
|
||||
BlockOrder.of(4, OrderType.DELETE));
|
||||
}
|
||||
|
||||
@Test
|
||||
void parseLine_singleOrder() {
|
||||
Line line = BsaDiffCreator.parseLine("testmark4,3008916894861");
|
||||
assertThat(line.label()).isEqualTo("testmark4");
|
||||
assertThat(line.orderIds()).containsExactly(3008916894861L);
|
||||
}
|
||||
|
||||
@Test
|
||||
void parseLine_multiOrder() {
|
||||
Line line = BsaDiffCreator.parseLine("xn--thnew-yorkinquirer-fxb,6927233432961;9314162579861");
|
||||
assertThat(line.label()).isEqualTo("xn--thnew-yorkinquirer-fxb");
|
||||
assertThat(line.orderIds()).containsExactly(6927233432961L, 9314162579861L);
|
||||
}
|
||||
|
||||
@Test
|
||||
void parseLine_invalidOrder() {
|
||||
assertThat(
|
||||
assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> BsaDiffCreator.parseLine("testmark4," + ORDER_ID_SENTINEL)))
|
||||
.hasMessageThat()
|
||||
.contains("Invalid order id");
|
||||
}
|
||||
|
||||
@Test
|
||||
void line_labelOrderPairs() {
|
||||
Line line = Line.of("a", ImmutableList.of(1L, 2L, 3L));
|
||||
assertThat(line.labelOrderPairs(new Canonicals<>()))
|
||||
.containsExactly(
|
||||
LabelOrderPair.of("a", 1L), LabelOrderPair.of("a", 2L), LabelOrderPair.of("a", 3L));
|
||||
}
|
||||
|
||||
@Test
|
||||
void canonicals_get() {
|
||||
Canonicals<Long> canonicals = new Canonicals<>();
|
||||
Long cvalue = canonicals.get(1L);
|
||||
assertThat(canonicals.get(1L)).isSameInstanceAs(cvalue);
|
||||
}
|
||||
}
|
98
core/src/test/java/google/registry/bsa/GcsClientTest.java
Normal file
98
core/src/test/java/google/registry/bsa/GcsClientTest.java
Normal file
|
@ -0,0 +1,98 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.cloud.storage.BlobId;
|
||||
import com.google.cloud.storage.contrib.nio.testing.LocalStorageHelper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.bsa.BlockListFetcher.LazyBlockList;
|
||||
import google.registry.bsa.api.BlockLabel;
|
||||
import google.registry.bsa.api.BlockLabel.LabelType;
|
||||
import google.registry.bsa.api.BlockOrder;
|
||||
import google.registry.bsa.api.BlockOrder.OrderType;
|
||||
import google.registry.gcs.GcsUtils;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.util.stream.Stream;
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
/** Unit tests for {@link GcsClient}. */
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
class GcsClientTest {
|
||||
|
||||
private GcsUtils gcsUtils = new GcsUtils(LocalStorageHelper.getOptions());
|
||||
|
||||
@Mock HttpsURLConnection connection;
|
||||
LazyBlockList lazyBlockList;
|
||||
GcsClient gcsClient;
|
||||
|
||||
@BeforeEach
|
||||
void setup() throws Exception {
|
||||
gcsClient = new GcsClient(gcsUtils, "my-bucket", "SHA-256");
|
||||
}
|
||||
|
||||
@Test
|
||||
void saveAndChecksumBlockList_success() throws Exception {
|
||||
String payload = "somedata\n";
|
||||
String payloadChecksum = "0737c8e591c68b93feccde50829aca86a80137547d8cfbe96bab6b20f8580c63";
|
||||
|
||||
when(connection.getInputStream())
|
||||
.thenReturn(new ByteArrayInputStream(("bsa-checksum\n" + payload).getBytes(UTF_8)));
|
||||
lazyBlockList = new LazyBlockList(BlockListType.BLOCK, connection);
|
||||
|
||||
ImmutableMap<BlockListType, String> checksums =
|
||||
gcsClient.saveAndChecksumBlockList("some-name", ImmutableList.of(lazyBlockList));
|
||||
assertThat(gcsUtils.existsAndNotEmpty(BlobId.of("my-bucket", "some-name/BLOCK.csv"))).isTrue();
|
||||
assertThat(checksums).containsExactly(BlockListType.BLOCK, payloadChecksum);
|
||||
assertThat(gcsClient.readBlockList("some-name", BlockListType.BLOCK))
|
||||
.containsExactly("somedata");
|
||||
}
|
||||
|
||||
@Test
|
||||
void readWrite_noData() throws Exception {
|
||||
gcsClient.writeOrderDiffs("job", Stream.of());
|
||||
assertThat(gcsClient.readOrderDiffs("job")).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void readWriteOrderDiffs_success() throws Exception {
|
||||
ImmutableList<BlockOrder> orders =
|
||||
ImmutableList.of(BlockOrder.of(1, OrderType.CREATE), BlockOrder.of(2, OrderType.DELETE));
|
||||
gcsClient.writeOrderDiffs("job", orders.stream());
|
||||
assertThat(gcsClient.readOrderDiffs("job")).containsExactlyElementsIn(orders);
|
||||
}
|
||||
|
||||
@Test
|
||||
void readWriteLabelDiffs_success() throws Exception {
|
||||
ImmutableList<BlockLabel> labels =
|
||||
ImmutableList.of(
|
||||
BlockLabel.of("1", LabelType.CREATE.CREATE, ImmutableSet.of()),
|
||||
BlockLabel.of("2", LabelType.NEW_ORDER_ASSOCIATION, ImmutableSet.of("JA")),
|
||||
BlockLabel.of("3", LabelType.DELETE, ImmutableSet.of("JA", "EXTENDED_LATIN")));
|
||||
gcsClient.writeLabelDiffs("job", labels.stream());
|
||||
assertThat(gcsClient.readLabelDiffs("job")).containsExactlyElementsIn(labels);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,133 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.bsa.ReservedDomainsUtils.getAllReservedDomainsInTld;
|
||||
import static google.registry.model.tld.Tld.TldState.GENERAL_AVAILABILITY;
|
||||
import static google.registry.model.tld.Tld.TldState.START_DATE_SUNRISE;
|
||||
import static google.registry.model.tld.label.ReservationType.ALLOWED_IN_SUNRISE;
|
||||
import static google.registry.model.tld.label.ReservationType.FULLY_BLOCKED;
|
||||
import static google.registry.model.tld.label.ReservationType.NAME_COLLISION;
|
||||
import static google.registry.model.tld.label.ReservationType.RESERVED_FOR_ANCHOR_TENANT;
|
||||
import static google.registry.model.tld.label.ReservationType.RESERVED_FOR_SPECIFIC_USE;
|
||||
import static google.registry.testing.DatabaseHelper.createTld;
|
||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.ImmutableSortedMap;
|
||||
import google.registry.model.tld.Tld;
|
||||
import google.registry.model.tld.label.ReservedList;
|
||||
import google.registry.model.tld.label.ReservedList.ReservedListEntry;
|
||||
import google.registry.model.tld.label.ReservedListDao;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationWithCoverageExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link ReservedDomainsUtils}. */
|
||||
class ReservedDomainsUtilsTest {
|
||||
|
||||
private final FakeClock fakeClock = new FakeClock();
|
||||
|
||||
@RegisterExtension
|
||||
JpaIntegrationWithCoverageExtension jpa =
|
||||
new JpaTestExtensions.Builder().withClock(fakeClock).buildIntegrationWithCoverageExtension();
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
ImmutableMap<String, ReservedListEntry> byType =
|
||||
ImmutableMap.of(
|
||||
"sunrise",
|
||||
ReservedListEntry.create("sunrise", ALLOWED_IN_SUNRISE, ""),
|
||||
"specific",
|
||||
ReservedListEntry.create("specific", RESERVED_FOR_SPECIFIC_USE, ""),
|
||||
"anchor",
|
||||
ReservedListEntry.create("anchor", RESERVED_FOR_ANCHOR_TENANT, ""),
|
||||
"fully",
|
||||
ReservedListEntry.create("fully", FULLY_BLOCKED, ""),
|
||||
"name",
|
||||
ReservedListEntry.create("name", NAME_COLLISION, ""));
|
||||
|
||||
ImmutableMap<String, ReservedListEntry> altList =
|
||||
ImmutableMap.of(
|
||||
"anchor",
|
||||
ReservedListEntry.create("anchor", RESERVED_FOR_ANCHOR_TENANT, ""),
|
||||
"somethingelse",
|
||||
ReservedListEntry.create("somethingelse", RESERVED_FOR_ANCHOR_TENANT, ""));
|
||||
|
||||
ReservedListDao.save(
|
||||
new ReservedList.Builder()
|
||||
.setName("testlist")
|
||||
.setCreationTimestamp(fakeClock.nowUtc())
|
||||
.setShouldPublish(false)
|
||||
.setReservedListMap(byType)
|
||||
.build());
|
||||
|
||||
ReservedListDao.save(
|
||||
new ReservedList.Builder()
|
||||
.setName("testlist2")
|
||||
.setCreationTimestamp(fakeClock.nowUtc())
|
||||
.setShouldPublish(false)
|
||||
.setReservedListMap(altList)
|
||||
.build());
|
||||
|
||||
createTld("tld");
|
||||
persistResource(
|
||||
Tld.get("tld")
|
||||
.asBuilder()
|
||||
.setTldStateTransitions(
|
||||
ImmutableSortedMap.of(
|
||||
fakeClock.nowUtc(), START_DATE_SUNRISE,
|
||||
fakeClock.nowUtc().plusMillis(1), GENERAL_AVAILABILITY))
|
||||
.setReservedListsByName(ImmutableSet.of("testlist"))
|
||||
.build());
|
||||
|
||||
createTld("tld2");
|
||||
persistResource(
|
||||
Tld.get("tld2")
|
||||
.asBuilder()
|
||||
.setReservedListsByName(ImmutableSet.of("testlist", "testlist2"))
|
||||
.build());
|
||||
}
|
||||
|
||||
@Test
|
||||
void enumerateReservedDomain_in_sunrise() {
|
||||
assertThat(getAllReservedDomainsInTld(Tld.get("tld"), fakeClock.nowUtc()))
|
||||
.containsExactly("specific.tld", "anchor.tld", "fully.tld");
|
||||
}
|
||||
|
||||
@Test
|
||||
void enumerateReservedDomain_after_sunrise() {
|
||||
fakeClock.advanceOneMilli();
|
||||
assertThat(getAllReservedDomainsInTld(Tld.get("tld"), fakeClock.nowUtc()))
|
||||
.containsExactly("sunrise.tld", "name.tld", "specific.tld", "anchor.tld", "fully.tld");
|
||||
}
|
||||
|
||||
@Test
|
||||
void enumerateReservedDomain_multiple_lists() {
|
||||
assertThat(getAllReservedDomainsInTld(Tld.get("tld2"), fakeClock.nowUtc()))
|
||||
.containsExactly(
|
||||
"somethingelse.tld2",
|
||||
"sunrise.tld2",
|
||||
"name.tld2",
|
||||
"specific.tld2",
|
||||
"anchor.tld2",
|
||||
"fully.tld2");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.bsa.api.BlockLabel.LabelType;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
/** Unit tests for {@link BlockLabel}. */
|
||||
class BlockLabelTest {
|
||||
|
||||
BlockLabel label;
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
label = BlockLabel.of("buy", LabelType.CREATE, ImmutableSet.of("JA", "EXTENDED_LATIN"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void serialize_success() {
|
||||
assertThat(label.serialize()).isEqualTo("buy,CREATE,EXTENDED_LATIN,JA");
|
||||
}
|
||||
|
||||
@Test
|
||||
void deserialize_success() {
|
||||
assertThat(BlockLabel.deserialize("buy,CREATE,EXTENDED_LATIN,JA")).isEqualTo(label);
|
||||
}
|
||||
|
||||
@Test
|
||||
void emptyIdns() {
|
||||
label = BlockLabel.of("buy", LabelType.CREATE, ImmutableSet.of());
|
||||
assertThat(label.serialize()).isEqualTo("buy,CREATE");
|
||||
assertThat(BlockLabel.deserialize("buy,CREATE")).isEqualTo(label);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
|
||||
import google.registry.bsa.api.BlockOrder.OrderType;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
/** Unit tests for {@link BlockOrder}. */
|
||||
class BlockOrderTest {
|
||||
|
||||
BlockOrder order;
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
order = BlockOrder.of(123, OrderType.CREATE);
|
||||
}
|
||||
|
||||
@Test
|
||||
void serialize_success() {
|
||||
assertThat(order.serialize()).isEqualTo("123,CREATE");
|
||||
}
|
||||
|
||||
@Test
|
||||
void deserialize_success() {
|
||||
assertThat(BlockOrder.deserialize("123,CREATE")).isEqualTo(order);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,141 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
||||
import static javax.servlet.http.HttpServletResponse.SC_OK;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyInt;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.doThrow;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.common.truth.Truth;
|
||||
import google.registry.keyring.api.Keyring;
|
||||
import google.registry.request.UrlConnectionService;
|
||||
import google.registry.testing.FakeClock;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.net.URL;
|
||||
import java.security.GeneralSecurityException;
|
||||
import javax.net.ssl.HttpsURLConnection;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
/** Unit tests for {@link BsaCredential}. */
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
class BsaCredentialTest {
|
||||
|
||||
private static final Duration AUTH_TOKEN_EXPIRY = Duration.standardMinutes(30);
|
||||
|
||||
@Mock OutputStream connectionOutputStream;
|
||||
@Mock HttpsURLConnection connection;
|
||||
@Mock UrlConnectionService connectionService;
|
||||
@Mock Keyring keyring;
|
||||
FakeClock clock = new FakeClock();
|
||||
BsaCredential credential;
|
||||
|
||||
@BeforeEach
|
||||
void setup() throws Exception {
|
||||
credential =
|
||||
new BsaCredential(connectionService, "https://authUrl", AUTH_TOKEN_EXPIRY, keyring, clock);
|
||||
}
|
||||
|
||||
void setupHttp() throws Exception {
|
||||
when(connectionService.createConnection(any(URL.class))).thenReturn(connection);
|
||||
when(connection.getOutputStream()).thenReturn(connectionOutputStream);
|
||||
when(keyring.getBsaApiKey()).thenReturn("bsaApiKey");
|
||||
}
|
||||
|
||||
@Test
|
||||
void getAuthToken_fetchesNew() throws Exception {
|
||||
credential = spy(credential);
|
||||
doReturn("a", "b", "c").when(credential).fetchNewAuthToken();
|
||||
assertThat(credential.getAuthToken()).isEqualTo("a");
|
||||
verify(credential, times(1)).fetchNewAuthToken();
|
||||
}
|
||||
|
||||
@Test
|
||||
void getAuthToken_useCached() throws Exception {
|
||||
credential = spy(credential);
|
||||
doReturn("a", "b", "c").when(credential).fetchNewAuthToken();
|
||||
assertThat(credential.getAuthToken()).isEqualTo("a");
|
||||
clock.advanceBy(AUTH_TOKEN_EXPIRY.minus(Duration.millis(1)));
|
||||
assertThat(credential.getAuthToken()).isEqualTo("a");
|
||||
verify(credential, times(1)).fetchNewAuthToken();
|
||||
}
|
||||
|
||||
@Test
|
||||
void getAuthToken_cacheExpires() throws Exception {
|
||||
credential = spy(credential);
|
||||
doReturn("a", "b", "c").when(credential).fetchNewAuthToken();
|
||||
assertThat(credential.getAuthToken()).isEqualTo("a");
|
||||
clock.advanceBy(AUTH_TOKEN_EXPIRY);
|
||||
assertThat(credential.getAuthToken()).isEqualTo("b");
|
||||
verify(credential, times(2)).fetchNewAuthToken();
|
||||
}
|
||||
|
||||
@Test
|
||||
void fetchNewAuthToken_success() throws Exception {
|
||||
setupHttp();
|
||||
when(connection.getResponseCode()).thenReturn(SC_OK);
|
||||
when(connection.getInputStream())
|
||||
.thenReturn(new ByteArrayInputStream("{\"id_token\": \"abc\"}".getBytes(UTF_8)));
|
||||
assertThat(credential.getAuthToken()).isEqualTo("abc");
|
||||
verify(connection, times(1)).setRequestMethod("POST");
|
||||
verify(connection, times(1))
|
||||
.setRequestProperty("Content-Type", "application/x-www-form-urlencoded");
|
||||
verify(connectionOutputStream, times(1))
|
||||
.write(eq("apiKey=bsaApiKey&space=BSA".getBytes(UTF_8)), anyInt(), anyInt());
|
||||
verify(connection, times(1)).disconnect();
|
||||
}
|
||||
|
||||
@Test
|
||||
void fetchNewAuthToken_whenStatusIsNotOK_throwsRetriableException() throws Exception {
|
||||
setupHttp();
|
||||
when(connection.getResponseCode()).thenReturn(202);
|
||||
Truth.assertThat(
|
||||
assertThrows(BsaException.class, () -> credential.getAuthToken()).isRetriable())
|
||||
.isTrue();
|
||||
}
|
||||
|
||||
@Test
|
||||
void fetchNewAuthToken_IOException_isRetriable() throws Exception {
|
||||
setupHttp();
|
||||
doThrow(new IOException()).when(connection).getResponseCode();
|
||||
assertThat(assertThrows(BsaException.class, () -> credential.getAuthToken()).isRetriable())
|
||||
.isTrue();
|
||||
}
|
||||
|
||||
@Test
|
||||
void fetchNewAuthToken_securityException_NotRetriable() throws Exception {
|
||||
doThrow(new GeneralSecurityException())
|
||||
.when(connectionService)
|
||||
.createConnection(any(URL.class));
|
||||
assertThat(assertThrows(BsaException.class, () -> credential.getAuthToken()).isRetriable())
|
||||
.isFalse();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,143 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.bsa.api.JsonSerializations.toCompletedOrdersReport;
|
||||
import static google.registry.bsa.api.JsonSerializations.toInProgressOrdersReport;
|
||||
import static google.registry.bsa.api.JsonSerializations.toUnblockableDomainsReport;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import google.registry.bsa.api.BlockOrder.OrderType;
|
||||
import google.registry.bsa.api.UnblockableDomain.Reason;
|
||||
import java.util.stream.Stream;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
|
||||
|
||||
/** Unit tests for {@link JsonSerializations}. */
|
||||
class JsonSerializationsTest {
|
||||
|
||||
@Test
|
||||
void toJson_inProgress_oneOrder() {
|
||||
// Below can be replaced with TextBlock in Java 15+
|
||||
// Put lines in an ImmutableList to preserve the look of the text both inside and outside IDE:
|
||||
// the parameter name blurbs inserted by IDE shift the indentation. Same for all tests below.
|
||||
String expected =
|
||||
Joiner.on('\n')
|
||||
.join(
|
||||
ImmutableList.of(
|
||||
"[",
|
||||
" {",
|
||||
" \"blockOrderId\": 1,",
|
||||
" \"status\": \"ActivationInProgress\"",
|
||||
" }",
|
||||
"]"));
|
||||
assertThat(toInProgressOrdersReport(Stream.of(BlockOrder.of(1, OrderType.CREATE))))
|
||||
.hasValue(expected);
|
||||
}
|
||||
|
||||
@Test
|
||||
void toJson_inProgress_multiOrders() {
|
||||
String expected =
|
||||
Joiner.on('\n')
|
||||
.join(
|
||||
ImmutableList.of(
|
||||
"[",
|
||||
" {",
|
||||
" \"blockOrderId\": 1,",
|
||||
" \"status\": \"ActivationInProgress\"",
|
||||
" },",
|
||||
" {",
|
||||
" \"blockOrderId\": 2,",
|
||||
" \"status\": \"ReleaseInProgress\"",
|
||||
" }",
|
||||
"]"));
|
||||
assertThat(
|
||||
toInProgressOrdersReport(
|
||||
Stream.of(BlockOrder.of(1, OrderType.CREATE), BlockOrder.of(2, OrderType.DELETE))))
|
||||
.hasValue(expected);
|
||||
}
|
||||
|
||||
@Test
|
||||
void toJson_completed_oneOrder() {
|
||||
String expected =
|
||||
Joiner.on('\n')
|
||||
.join(
|
||||
ImmutableList.of(
|
||||
"[",
|
||||
" {",
|
||||
" \"blockOrderId\": 1,",
|
||||
" \"status\": \"Active\"",
|
||||
" }",
|
||||
"]"));
|
||||
assertThat(toCompletedOrdersReport(Stream.of(BlockOrder.of(1, OrderType.CREATE))))
|
||||
.hasValue(expected);
|
||||
}
|
||||
|
||||
@Test
|
||||
void toJson_completed_multiOrders() {
|
||||
String expected =
|
||||
Joiner.on('\n')
|
||||
.join(
|
||||
ImmutableList.of(
|
||||
"[",
|
||||
" {",
|
||||
" \"blockOrderId\": 1,",
|
||||
" \"status\": \"Active\"",
|
||||
" },",
|
||||
" {",
|
||||
" \"blockOrderId\": 2,",
|
||||
" \"status\": \"Closed\"",
|
||||
" }",
|
||||
"]"));
|
||||
assertThat(
|
||||
toCompletedOrdersReport(
|
||||
Stream.of(BlockOrder.of(1, OrderType.CREATE), BlockOrder.of(2, OrderType.DELETE))))
|
||||
.hasValue(expected);
|
||||
}
|
||||
|
||||
@Test
|
||||
void toJson_UnblockableDomains_empty() {
|
||||
assertThat(toUnblockableDomainsReport(Stream.<UnblockableDomain>of())).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void toJson_UnblockableDomains_notEmpty() {
|
||||
String expected =
|
||||
Joiner.on('\n')
|
||||
.join(
|
||||
ImmutableList.of(
|
||||
"{",
|
||||
" \"invalid\": [",
|
||||
" \"b.app\"",
|
||||
" ],",
|
||||
" \"registered\": [",
|
||||
" \"a.ing\",",
|
||||
" \"d.page\"",
|
||||
" ],",
|
||||
" \"reserved\": [",
|
||||
" \"c.dev\"",
|
||||
" ]",
|
||||
"}"));
|
||||
assertThat(
|
||||
toUnblockableDomainsReport(
|
||||
Stream.of(
|
||||
UnblockableDomain.of("a.ing", Reason.REGISTERED),
|
||||
UnblockableDomain.of("b.app", Reason.INVALID),
|
||||
UnblockableDomain.of("c.dev", Reason.RESERVED),
|
||||
UnblockableDomain.of("d.page", Reason.REGISTERED))))
|
||||
.hasValue(expected);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.api;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
|
||||
import google.registry.bsa.api.UnblockableDomain.Reason;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
/** Unit tests for {@link UnblockableDomain}. */
|
||||
class UnblockableDomainTest {
|
||||
|
||||
UnblockableDomain unit;
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
unit = UnblockableDomain.of("buy.app", Reason.REGISTERED);
|
||||
}
|
||||
|
||||
@Test
|
||||
void serialize_success() {
|
||||
assertThat(unit.serialize()).isEqualTo("buy.app,REGISTERED");
|
||||
}
|
||||
|
||||
@Test
|
||||
void deserialize_success() {
|
||||
assertThat(UnblockableDomain.deserialize("buy.app,REGISTERED")).isEqualTo(unit);
|
||||
}
|
||||
|
||||
@Test
|
||||
void alt_of() {
|
||||
assertThat(UnblockableDomain.of("buy", "app", Reason.REGISTERED)).isEqualTo(unit);
|
||||
}
|
||||
}
|
|
@ -15,7 +15,7 @@
|
|||
package google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.bsa.persistence.BsaDomainRefresh.Stage.MAKE_DIFF;
|
||||
import static google.registry.bsa.RefreshStage.CHECK_FOR_CHANGES;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
|
@ -41,14 +41,13 @@ public class BsaDomainRefreshTest {
|
|||
tm().transact(() -> tm().getEntityManager().merge(new BsaDomainRefresh()));
|
||||
assertThat(persisted.jobId).isNotNull();
|
||||
assertThat(persisted.creationTime.getTimestamp()).isEqualTo(fakeClock.nowUtc());
|
||||
assertThat(persisted.stage).isEqualTo(MAKE_DIFF);
|
||||
assertThat(persisted.stage).isEqualTo(CHECK_FOR_CHANGES);
|
||||
}
|
||||
|
||||
@Test
|
||||
void loadJobByKey() {
|
||||
BsaDomainRefresh persisted =
|
||||
tm().transact(() -> tm().getEntityManager().merge(new BsaDomainRefresh()));
|
||||
assertThat(tm().transact(() -> tm().loadByKey(BsaDomainRefresh.vKey(persisted))))
|
||||
.isEqualTo(persisted);
|
||||
assertThat(tm().transact(() -> tm().loadByKey(persisted.vKey()))).isEqualTo(persisted);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -15,14 +15,14 @@
|
|||
package google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.bsa.BlockList.BLOCK;
|
||||
import static google.registry.bsa.BlockList.BLOCK_PLUS;
|
||||
import static google.registry.bsa.DownloadStage.DOWNLOAD;
|
||||
import static google.registry.bsa.BlockListType.BLOCK;
|
||||
import static google.registry.bsa.BlockListType.BLOCK_PLUS;
|
||||
import static google.registry.bsa.DownloadStage.DOWNLOAD_BLOCK_LISTS;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import google.registry.bsa.BlockList;
|
||||
import google.registry.bsa.BlockListType;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationWithCoverageExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
|
@ -33,7 +33,7 @@ import org.junit.jupiter.api.extension.RegisterExtension;
|
|||
/** Unit test for {@link BsaDownload}. */
|
||||
public class BsaDownloadTest {
|
||||
|
||||
protected FakeClock fakeClock = new FakeClock(DateTime.now(UTC));
|
||||
FakeClock fakeClock = new FakeClock(DateTime.now(UTC));
|
||||
|
||||
@RegisterExtension
|
||||
final JpaIntegrationWithCoverageExtension jpa =
|
||||
|
@ -44,7 +44,7 @@ public class BsaDownloadTest {
|
|||
BsaDownload persisted = tm().transact(() -> tm().getEntityManager().merge(new BsaDownload()));
|
||||
assertThat(persisted.jobId).isNotNull();
|
||||
assertThat(persisted.creationTime.getTimestamp()).isEqualTo(fakeClock.nowUtc());
|
||||
assertThat(persisted.stage).isEqualTo(DOWNLOAD);
|
||||
assertThat(persisted.stage).isEqualTo(DOWNLOAD_BLOCK_LISTS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -58,7 +58,7 @@ public class BsaDownloadTest {
|
|||
void checksums() {
|
||||
BsaDownload job = new BsaDownload();
|
||||
assertThat(job.getChecksums()).isEmpty();
|
||||
ImmutableMap<BlockList, String> checksums = ImmutableMap.of(BLOCK, "a", BLOCK_PLUS, "b");
|
||||
ImmutableMap<BlockListType, String> checksums = ImmutableMap.of(BLOCK, "a", BLOCK_PLUS, "b");
|
||||
job.setChecksums(checksums);
|
||||
assertThat(job.getChecksums()).isEqualTo(checksums);
|
||||
assertThat(job.blockListChecksums).isEqualTo("BLOCK=a,BLOCK_PLUS=b");
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.junit.jupiter.api.extension.RegisterExtension;
|
|||
/** Unit tests for {@link BsaLabel}. */
|
||||
public class BsaLabelTest {
|
||||
|
||||
protected FakeClock fakeClock = new FakeClock(DateTime.now(UTC));
|
||||
FakeClock fakeClock = new FakeClock(DateTime.now(UTC));
|
||||
|
||||
@RegisterExtension
|
||||
final JpaIntegrationWithCoverageExtension jpa =
|
||||
|
|
|
@ -20,7 +20,8 @@ import static google.registry.persistence.transaction.TransactionManagerFactory.
|
|||
import static org.joda.time.DateTimeZone.UTC;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
import google.registry.bsa.persistence.BsaDomainInUse.Reason;
|
||||
import google.registry.bsa.api.UnblockableDomain;
|
||||
import google.registry.bsa.persistence.BsaUnblockableDomain.Reason;
|
||||
import google.registry.persistence.transaction.DatabaseException;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationWithCoverageExtension;
|
||||
|
@ -29,10 +30,10 @@ import org.joda.time.DateTime;
|
|||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link BsaDomainInUse}. */
|
||||
public class BsaDomainInUseTest {
|
||||
/** Unit tests for {@link BsaUnblockableDomain}. */
|
||||
public class BsaUnblockableDomainTest {
|
||||
|
||||
protected FakeClock fakeClock = new FakeClock(DateTime.now(UTC));
|
||||
FakeClock fakeClock = new FakeClock(DateTime.now(UTC));
|
||||
|
||||
@RegisterExtension
|
||||
final JpaIntegrationWithCoverageExtension jpa =
|
||||
|
@ -41,9 +42,9 @@ public class BsaDomainInUseTest {
|
|||
@Test
|
||||
void persist() {
|
||||
tm().transact(() -> tm().put(new BsaLabel("label", fakeClock.nowUtc())));
|
||||
tm().transact(() -> tm().put(new BsaDomainInUse("label", "tld", Reason.REGISTERED)));
|
||||
BsaDomainInUse persisted =
|
||||
tm().transact(() -> tm().loadByKey(BsaDomainInUse.vKey("label", "tld")));
|
||||
tm().transact(() -> tm().put(new BsaUnblockableDomain("label", "tld", Reason.REGISTERED)));
|
||||
BsaUnblockableDomain persisted =
|
||||
tm().transact(() -> tm().loadByKey(BsaUnblockableDomain.vKey("label", "tld")));
|
||||
assertThat(persisted.label).isEqualTo("label");
|
||||
assertThat(persisted.tld).isEqualTo("tld");
|
||||
assertThat(persisted.reason).isEqualTo(Reason.REGISTERED);
|
||||
|
@ -52,11 +53,13 @@ public class BsaDomainInUseTest {
|
|||
@Test
|
||||
void cascadeDeletion() {
|
||||
tm().transact(() -> tm().put(new BsaLabel("label", fakeClock.nowUtc())));
|
||||
tm().transact(() -> tm().put(new BsaDomainInUse("label", "tld", Reason.REGISTERED)));
|
||||
assertThat(tm().transact(() -> tm().loadByKeyIfPresent(BsaDomainInUse.vKey("label", "tld"))))
|
||||
tm().transact(() -> tm().put(new BsaUnblockableDomain("label", "tld", Reason.REGISTERED)));
|
||||
assertThat(
|
||||
tm().transact(() -> tm().loadByKeyIfPresent(BsaUnblockableDomain.vKey("label", "tld"))))
|
||||
.isPresent();
|
||||
tm().transact(() -> tm().delete(BsaLabel.vKey("label")));
|
||||
assertThat(tm().transact(() -> tm().loadByKeyIfPresent(BsaDomainInUse.vKey("label", "tld"))))
|
||||
assertThat(
|
||||
tm().transact(() -> tm().loadByKeyIfPresent(BsaUnblockableDomain.vKey("label", "tld"))))
|
||||
.isEmpty();
|
||||
}
|
||||
|
||||
|
@ -67,8 +70,25 @@ public class BsaDomainInUseTest {
|
|||
DatabaseException.class,
|
||||
() ->
|
||||
tm().transact(
|
||||
() -> tm().put(new BsaDomainInUse("label", "tld", Reason.REGISTERED)))))
|
||||
() ->
|
||||
tm().put(
|
||||
new BsaUnblockableDomain(
|
||||
"label", "tld", Reason.REGISTERED)))))
|
||||
.hasMessageThat()
|
||||
.contains("violates foreign key constraint");
|
||||
}
|
||||
|
||||
@Test
|
||||
void reason_convertibleToApiClass() {
|
||||
for (BsaUnblockableDomain.Reason reason : BsaUnblockableDomain.Reason.values()) {
|
||||
try {
|
||||
UnblockableDomain.Reason.valueOf(reason.name());
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
"Missing enum name [%s] in %s",
|
||||
reason.name(), BsaUnblockableDomain.Reason.class.getName()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,162 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.bsa.BsaTransactions.bsaTransact;
|
||||
import static google.registry.bsa.persistence.BsaLabelTestingUtils.persistBsaLabel;
|
||||
import static google.registry.model.tld.label.ReservationType.RESERVED_FOR_SPECIFIC_USE;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.testing.DatabaseHelper.createTld;
|
||||
import static google.registry.testing.DatabaseHelper.newDomain;
|
||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.bsa.api.UnblockableDomain;
|
||||
import google.registry.bsa.api.UnblockableDomainChange;
|
||||
import google.registry.bsa.persistence.BsaUnblockableDomain.Reason;
|
||||
import google.registry.model.tld.Tld;
|
||||
import google.registry.model.tld.label.ReservedList;
|
||||
import google.registry.model.tld.label.ReservedList.ReservedListEntry;
|
||||
import google.registry.model.tld.label.ReservedListDao;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationWithCoverageExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
import java.util.Optional;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DomainsRefresher}. */
|
||||
public class DomainRefresherTest {
|
||||
|
||||
FakeClock fakeClock = new FakeClock(DateTime.parse("2023-11-09T02:08:57.880Z"));
|
||||
|
||||
@RegisterExtension
|
||||
final JpaIntegrationWithCoverageExtension jpa =
|
||||
new JpaTestExtensions.Builder().withClock(fakeClock).buildIntegrationWithCoverageExtension();
|
||||
|
||||
DomainsRefresher refresher;
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
createTld("tld");
|
||||
persistResource(
|
||||
Tld.get("tld")
|
||||
.asBuilder()
|
||||
.setBsaEnrollStartTime(Optional.of(fakeClock.nowUtc().minusMillis(1)))
|
||||
.build());
|
||||
refresher = new DomainsRefresher(START_OF_TIME, fakeClock.nowUtc(), Duration.ZERO, 100);
|
||||
}
|
||||
|
||||
@Test
|
||||
void staleUnblockableRemoved_wasRegistered() {
|
||||
persistBsaLabel("label", fakeClock.nowUtc().minus(Duration.standardDays(1)));
|
||||
tm().transact(() -> tm().insert(BsaUnblockableDomain.of("label.tld", Reason.REGISTERED)));
|
||||
assertThat(bsaTransact(refresher::refreshStaleUnblockables))
|
||||
.containsExactly(
|
||||
UnblockableDomainChange.ofDeleted(
|
||||
UnblockableDomain.of("label.tld", UnblockableDomain.Reason.REGISTERED)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void staleUnblockableRemoved_wasReserved() {
|
||||
persistBsaLabel("label", fakeClock.nowUtc().minus(Duration.standardDays(1)));
|
||||
tm().transact(() -> tm().insert(BsaUnblockableDomain.of("label.tld", Reason.RESERVED)));
|
||||
assertThat(bsaTransact(refresher::refreshStaleUnblockables))
|
||||
.containsExactly(
|
||||
UnblockableDomainChange.ofDeleted(
|
||||
UnblockableDomain.of("label.tld", UnblockableDomain.Reason.RESERVED)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void newUnblockableAdded_isRegistered() {
|
||||
persistResource(newDomain("label.tld"));
|
||||
persistBsaLabel("label", fakeClock.nowUtc().minus(Duration.standardDays(1)));
|
||||
assertThat(bsaTransact(refresher::getNewUnblockables))
|
||||
.containsExactly(
|
||||
UnblockableDomainChange.ofNew(
|
||||
UnblockableDomain.of("label.tld", UnblockableDomain.Reason.REGISTERED)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void newUnblockableAdded_isReserved() {
|
||||
persistBsaLabel("label", fakeClock.nowUtc().minus(Duration.standardDays(1)));
|
||||
setReservedList("label");
|
||||
assertThat(bsaTransact(refresher::getNewUnblockables))
|
||||
.containsExactly(
|
||||
UnblockableDomainChange.ofNew(
|
||||
UnblockableDomain.of("label.tld", UnblockableDomain.Reason.RESERVED)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void staleUnblockableDowngraded_registeredToReserved() {
|
||||
persistBsaLabel("label", fakeClock.nowUtc().minus(Duration.standardDays(1)));
|
||||
setReservedList("label");
|
||||
tm().transact(() -> tm().insert(BsaUnblockableDomain.of("label.tld", Reason.REGISTERED)));
|
||||
|
||||
assertThat(bsaTransact(refresher::refreshStaleUnblockables))
|
||||
.containsExactly(
|
||||
UnblockableDomainChange.ofChanged(
|
||||
UnblockableDomain.of("label.tld", UnblockableDomain.Reason.REGISTERED),
|
||||
UnblockableDomain.Reason.RESERVED));
|
||||
}
|
||||
|
||||
@Test
|
||||
void staleUnblockableUpgraded_reservedToRegisteredButNotReserved() {
|
||||
persistBsaLabel("label", fakeClock.nowUtc().minus(Duration.standardDays(1)));
|
||||
tm().transact(() -> tm().insert(BsaUnblockableDomain.of("label.tld", Reason.RESERVED)));
|
||||
|
||||
persistResource(newDomain("label.tld"));
|
||||
assertThat(bsaTransact(refresher::refreshStaleUnblockables))
|
||||
.containsExactly(
|
||||
UnblockableDomainChange.ofChanged(
|
||||
UnblockableDomain.of("label.tld", UnblockableDomain.Reason.RESERVED),
|
||||
UnblockableDomain.Reason.REGISTERED));
|
||||
}
|
||||
|
||||
@Test
|
||||
void staleUnblockableUpgraded_wasReserved_isReservedAndRegistered() {
|
||||
persistBsaLabel("label", fakeClock.nowUtc().minus(Duration.standardDays(1)));
|
||||
setReservedList("label");
|
||||
tm().transact(() -> tm().insert(BsaUnblockableDomain.of("label.tld", Reason.RESERVED)));
|
||||
|
||||
persistResource(newDomain("label.tld"));
|
||||
assertThat(bsaTransact(refresher::refreshStaleUnblockables))
|
||||
.containsExactly(
|
||||
UnblockableDomainChange.ofChanged(
|
||||
UnblockableDomain.of("label.tld", UnblockableDomain.Reason.RESERVED),
|
||||
UnblockableDomain.Reason.REGISTERED));
|
||||
}
|
||||
|
||||
private void setReservedList(String label) {
|
||||
ImmutableMap<String, ReservedListEntry> reservedNameMap =
|
||||
ImmutableMap.of(label, ReservedListEntry.create(label, RESERVED_FOR_SPECIFIC_USE, ""));
|
||||
|
||||
ReservedListDao.save(
|
||||
new ReservedList.Builder()
|
||||
.setName("testlist")
|
||||
.setCreationTimestamp(fakeClock.nowUtc())
|
||||
.setShouldPublish(false)
|
||||
.setReservedListMap(reservedNameMap)
|
||||
.build());
|
||||
persistResource(
|
||||
Tld.get("tld").asBuilder().setReservedListsByName(ImmutableSet.of("testlist")).build());
|
||||
}
|
||||
}
|
|
@ -16,21 +16,21 @@ package google.registry.bsa.persistence;
|
|||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.bsa.DownloadStage.CHECKSUMS_NOT_MATCH;
|
||||
import static google.registry.bsa.DownloadStage.CHECKSUMS_DO_NOT_MATCH;
|
||||
import static google.registry.bsa.DownloadStage.DONE;
|
||||
import static google.registry.bsa.DownloadStage.DOWNLOAD;
|
||||
import static google.registry.bsa.DownloadStage.MAKE_DIFF;
|
||||
import static google.registry.bsa.DownloadStage.DOWNLOAD_BLOCK_LISTS;
|
||||
import static google.registry.bsa.DownloadStage.MAKE_ORDER_AND_LABEL_DIFF;
|
||||
import static google.registry.bsa.DownloadStage.NOP;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
import static org.joda.time.Duration.standardDays;
|
||||
import static org.joda.time.Duration.standardMinutes;
|
||||
import static org.joda.time.Duration.standardSeconds;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.bsa.BlockList;
|
||||
import google.registry.bsa.BlockListType;
|
||||
import google.registry.bsa.DownloadStage;
|
||||
import google.registry.bsa.RefreshStage;
|
||||
import google.registry.bsa.persistence.DownloadSchedule.CompletedJob;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationWithCoverageExtension;
|
||||
|
@ -44,12 +44,12 @@ import org.junit.jupiter.api.Test;
|
|||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link DownloadScheduler} */
|
||||
public class DownloadSchedulerTest {
|
||||
class DownloadSchedulerTest {
|
||||
|
||||
static final Duration DOWNLOAD_INTERVAL = standardMinutes(30);
|
||||
static final Duration MAX_NOP_INTERVAL = standardDays(1);
|
||||
|
||||
protected FakeClock fakeClock = new FakeClock(DateTime.now(UTC));
|
||||
FakeClock fakeClock = new FakeClock(DateTime.parse("2023-11-09T02:08:57.880Z"));
|
||||
|
||||
@RegisterExtension
|
||||
final JpaIntegrationWithCoverageExtension jpa =
|
||||
|
@ -64,7 +64,7 @@ public class DownloadSchedulerTest {
|
|||
|
||||
@AfterEach
|
||||
void dbCheck() {
|
||||
ImmutableSet<DownloadStage> terminalStages = ImmutableSet.of(DONE, NOP, CHECKSUMS_NOT_MATCH);
|
||||
ImmutableSet<DownloadStage> terminalStages = ImmutableSet.of(DONE, NOP, CHECKSUMS_DO_NOT_MATCH);
|
||||
assertThat(
|
||||
tm().transact(
|
||||
() ->
|
||||
|
@ -82,20 +82,21 @@ public class DownloadSchedulerTest {
|
|||
assertThat(scheduleOptional).isPresent();
|
||||
DownloadSchedule schedule = scheduleOptional.get();
|
||||
assertThat(schedule.latestCompleted()).isEmpty();
|
||||
assertThat(schedule.jobName()).isEqualTo(fakeClock.nowUtc().toString());
|
||||
assertThat(schedule.stage()).isEqualTo(DownloadStage.DOWNLOAD);
|
||||
assertThat(schedule.jobName()).isEqualTo("2023-11-09t020857.880z");
|
||||
assertThat(schedule.stage()).isEqualTo(DownloadStage.DOWNLOAD_BLOCK_LISTS);
|
||||
assertThat(schedule.alwaysDownload()).isTrue();
|
||||
}
|
||||
|
||||
@Test
|
||||
void oneInProgressJob() {
|
||||
BsaDownload inProgressJob = insertOneJobAndAdvanceClock(MAKE_DIFF);
|
||||
BsaDownload inProgressJob = insertOneJobAndAdvanceClock(MAKE_ORDER_AND_LABEL_DIFF);
|
||||
Optional<DownloadSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isPresent();
|
||||
DownloadSchedule schedule = scheduleOptional.get();
|
||||
assertThat(schedule.jobId()).isEqualTo(inProgressJob.jobId);
|
||||
assertThat(schedule.jobCreationTime()).isEqualTo(inProgressJob.getCreationTime());
|
||||
assertThat(schedule.jobName()).isEqualTo(inProgressJob.getJobName());
|
||||
assertThat(schedule.stage()).isEqualTo(MAKE_DIFF);
|
||||
assertThat(schedule.stage()).isEqualTo(MAKE_ORDER_AND_LABEL_DIFF);
|
||||
assertThat(schedule.latestCompleted()).isEmpty();
|
||||
assertThat(schedule.alwaysDownload()).isTrue();
|
||||
}
|
||||
|
@ -103,13 +104,14 @@ public class DownloadSchedulerTest {
|
|||
@Test
|
||||
void oneInProgressJobOneCompletedJob() {
|
||||
BsaDownload completed = insertOneJobAndAdvanceClock(DONE);
|
||||
BsaDownload inProgressJob = insertOneJobAndAdvanceClock(MAKE_DIFF);
|
||||
BsaDownload inProgressJob = insertOneJobAndAdvanceClock(MAKE_ORDER_AND_LABEL_DIFF);
|
||||
Optional<DownloadSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isPresent();
|
||||
DownloadSchedule schedule = scheduleOptional.get();
|
||||
assertThat(schedule.jobId()).isEqualTo(inProgressJob.jobId);
|
||||
assertThat(schedule.jobCreationTime()).isEqualTo(inProgressJob.getCreationTime());
|
||||
assertThat(schedule.jobName()).isEqualTo(inProgressJob.getJobName());
|
||||
assertThat(schedule.stage()).isEqualTo(MAKE_DIFF);
|
||||
assertThat(schedule.stage()).isEqualTo(MAKE_ORDER_AND_LABEL_DIFF);
|
||||
assertThat(schedule.alwaysDownload()).isFalse();
|
||||
assertThat(schedule.latestCompleted()).isPresent();
|
||||
CompletedJob lastCompleted = schedule.latestCompleted().get();
|
||||
|
@ -130,7 +132,7 @@ public class DownloadSchedulerTest {
|
|||
Optional<DownloadSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isPresent();
|
||||
DownloadSchedule schedule = scheduleOptional.get();
|
||||
assertThat(schedule.stage()).isEqualTo(DOWNLOAD);
|
||||
assertThat(schedule.stage()).isEqualTo(DOWNLOAD_BLOCK_LISTS);
|
||||
assertThat(schedule.alwaysDownload()).isFalse();
|
||||
assertThat(schedule.latestCompleted()).isPresent();
|
||||
CompletedJob completedJob = schedule.latestCompleted().get();
|
||||
|
@ -164,26 +166,26 @@ public class DownloadSchedulerTest {
|
|||
|
||||
@Test
|
||||
void loadRecentProcessedJobs_noneExists() {
|
||||
assertThat(tm().transact(() -> scheduler.loadRecentProcessedJobs())).isEmpty();
|
||||
assertThat(tm().transact(() -> scheduler.fetchTwoMostRecentDownloads())).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void loadRecentProcessedJobs_nopJobsOnly() {
|
||||
insertOneJobAndAdvanceClock(DownloadStage.NOP);
|
||||
insertOneJobAndAdvanceClock(DownloadStage.CHECKSUMS_NOT_MATCH);
|
||||
assertThat(tm().transact(() -> scheduler.loadRecentProcessedJobs())).isEmpty();
|
||||
insertOneJobAndAdvanceClock(DownloadStage.CHECKSUMS_DO_NOT_MATCH);
|
||||
assertThat(tm().transact(() -> scheduler.fetchTwoMostRecentDownloads())).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void loadRecentProcessedJobs_oneInProgressJob() {
|
||||
BsaDownload job = insertOneJobAndAdvanceClock(MAKE_DIFF);
|
||||
assertThat(tm().transact(() -> scheduler.loadRecentProcessedJobs())).containsExactly(job);
|
||||
BsaDownload job = insertOneJobAndAdvanceClock(MAKE_ORDER_AND_LABEL_DIFF);
|
||||
assertThat(tm().transact(() -> scheduler.fetchTwoMostRecentDownloads())).containsExactly(job);
|
||||
}
|
||||
|
||||
@Test
|
||||
void loadRecentProcessedJobs_oneDoneJob() {
|
||||
BsaDownload job = insertOneJobAndAdvanceClock(DONE);
|
||||
assertThat(tm().transact(() -> scheduler.loadRecentProcessedJobs())).containsExactly(job);
|
||||
assertThat(tm().transact(() -> scheduler.fetchTwoMostRecentDownloads())).containsExactly(job);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -192,17 +194,35 @@ public class DownloadSchedulerTest {
|
|||
insertOneJobAndAdvanceClock(DownloadStage.DONE);
|
||||
BsaDownload completed = insertOneJobAndAdvanceClock(DownloadStage.DONE);
|
||||
insertOneJobAndAdvanceClock(DownloadStage.NOP);
|
||||
insertOneJobAndAdvanceClock(DownloadStage.CHECKSUMS_NOT_MATCH);
|
||||
BsaDownload inprogress = insertOneJobAndAdvanceClock(DownloadStage.APPLY_DIFF);
|
||||
assertThat(tm().transact(() -> scheduler.loadRecentProcessedJobs()))
|
||||
insertOneJobAndAdvanceClock(DownloadStage.CHECKSUMS_DO_NOT_MATCH);
|
||||
BsaDownload inprogress = insertOneJobAndAdvanceClock(DownloadStage.APPLY_ORDER_AND_LABEL_DIFF);
|
||||
assertThat(tm().transact(() -> scheduler.fetchTwoMostRecentDownloads()))
|
||||
.containsExactly(inprogress, completed)
|
||||
.inOrder();
|
||||
}
|
||||
|
||||
@Test
|
||||
void ongoingRefresh_noNewSchedule() {
|
||||
insertOneJobAndAdvanceClock(DONE);
|
||||
tm().transact(() -> tm().insert(new BsaDomainRefresh()));
|
||||
fakeClock.advanceBy(DOWNLOAD_INTERVAL);
|
||||
Optional<DownloadSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void doneRefresh_noNewSchedule() {
|
||||
insertOneJobAndAdvanceClock(DONE);
|
||||
tm().transact(() -> tm().insert(new BsaDomainRefresh().setStage(RefreshStage.DONE)));
|
||||
fakeClock.advanceBy(DOWNLOAD_INTERVAL);
|
||||
Optional<DownloadSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isPresent();
|
||||
}
|
||||
|
||||
private BsaDownload insertOneJobAndAdvanceClock(DownloadStage stage) {
|
||||
BsaDownload job = new BsaDownload();
|
||||
job.setStage(stage);
|
||||
job.setChecksums(ImmutableMap.of(BlockList.BLOCK, "1", BlockList.BLOCK_PLUS, "2"));
|
||||
job.setChecksums(ImmutableMap.of(BlockListType.BLOCK, "1", BlockListType.BLOCK_PLUS, "2"));
|
||||
tm().transact(() -> tm().insert(job));
|
||||
fakeClock.advanceOneMilli();
|
||||
return job;
|
||||
|
|
|
@ -0,0 +1,179 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.bsa.persistence.LabelDiffUpdates.applyLabelDiff;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.testing.DatabaseHelper.createTld;
|
||||
import static google.registry.testing.DatabaseHelper.persistActiveDomain;
|
||||
import static google.registry.tldconfig.idn.IdnTableEnum.UNCONFUSABLE_LATIN;
|
||||
import static google.registry.util.DateTimeUtils.START_OF_TIME;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import google.registry.bsa.IdnChecker;
|
||||
import google.registry.bsa.api.BlockLabel;
|
||||
import google.registry.bsa.api.BlockLabel.LabelType;
|
||||
import google.registry.bsa.api.UnblockableDomain;
|
||||
import google.registry.bsa.persistence.BsaUnblockableDomain.Reason;
|
||||
import google.registry.model.tld.Tld;
|
||||
import google.registry.model.tld.label.ReservationType;
|
||||
import google.registry.model.tld.label.ReservedList;
|
||||
import google.registry.model.tld.label.ReservedList.ReservedListEntry;
|
||||
import google.registry.model.tld.label.ReservedListDao;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationWithCoverageExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
import java.util.Optional;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
/** Unit tests for {@link LabelDiffUpdates}. */
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
class LabelDiffUpdatesTest {
|
||||
|
||||
FakeClock fakeClock = new FakeClock(DateTime.parse("2023-11-09T02:08:57.880Z"));
|
||||
|
||||
@RegisterExtension
|
||||
final JpaIntegrationWithCoverageExtension jpa =
|
||||
new JpaTestExtensions.Builder().withClock(fakeClock).buildIntegrationWithCoverageExtension();
|
||||
|
||||
@Mock IdnChecker idnChecker;
|
||||
@Mock DownloadSchedule schedule;
|
||||
|
||||
Tld app;
|
||||
Tld dev;
|
||||
Tld page;
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
Tld tld = createTld("app");
|
||||
tm().transact(
|
||||
() ->
|
||||
tm().put(
|
||||
tld.asBuilder()
|
||||
.setBsaEnrollStartTime(Optional.of(START_OF_TIME))
|
||||
.setIdnTables(ImmutableSet.of(UNCONFUSABLE_LATIN))
|
||||
.build()));
|
||||
app = tm().transact(() -> tm().loadByEntity(tld));
|
||||
dev = createTld("dev");
|
||||
page = createTld("page");
|
||||
}
|
||||
|
||||
@Test
|
||||
void applyLabelDiffs_delete() {
|
||||
tm().transact(
|
||||
() -> {
|
||||
tm().insert(new BsaLabel("label", fakeClock.nowUtc()));
|
||||
tm().insert(new BsaUnblockableDomain("label", "app", Reason.REGISTERED));
|
||||
});
|
||||
when(idnChecker.getSupportingTlds(any())).thenReturn(ImmutableSet.of(app));
|
||||
|
||||
ImmutableList<UnblockableDomain> unblockableDomains =
|
||||
applyLabelDiff(
|
||||
ImmutableList.of(BlockLabel.of("label", LabelType.DELETE, ImmutableSet.of())),
|
||||
idnChecker,
|
||||
schedule,
|
||||
fakeClock.nowUtc());
|
||||
assertThat(unblockableDomains).isEmpty();
|
||||
assertThat(tm().transact(() -> tm().loadByKeyIfPresent(BsaLabel.vKey("label")))).isEmpty();
|
||||
assertThat(
|
||||
tm().transact(() -> tm().loadByKeyIfPresent(BsaUnblockableDomain.vKey("label", "app"))))
|
||||
.isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void applyLabelDiffs_newAssociationOfLabelToOrder() {
|
||||
tm().transact(
|
||||
() -> {
|
||||
tm().insert(new BsaLabel("label", fakeClock.nowUtc()));
|
||||
tm().insert(new BsaUnblockableDomain("label", "app", Reason.REGISTERED));
|
||||
});
|
||||
when(idnChecker.getSupportingTlds(any())).thenReturn(ImmutableSet.of(app));
|
||||
when(idnChecker.getForbiddingTlds(any()))
|
||||
.thenReturn(Sets.difference(ImmutableSet.of(dev), ImmutableSet.of()).immutableCopy());
|
||||
|
||||
ImmutableList<UnblockableDomain> unblockableDomains =
|
||||
applyLabelDiff(
|
||||
ImmutableList.of(
|
||||
BlockLabel.of("label", LabelType.NEW_ORDER_ASSOCIATION, ImmutableSet.of())),
|
||||
idnChecker,
|
||||
schedule,
|
||||
fakeClock.nowUtc());
|
||||
assertThat(unblockableDomains)
|
||||
.containsExactly(
|
||||
UnblockableDomain.of("label.app", UnblockableDomain.Reason.REGISTERED),
|
||||
UnblockableDomain.of("label.dev", UnblockableDomain.Reason.INVALID));
|
||||
assertThat(tm().transact(() -> tm().loadByKeyIfPresent(BsaLabel.vKey("label")))).isPresent();
|
||||
assertThat(
|
||||
tm().transact(() -> tm().loadByKeyIfPresent(BsaUnblockableDomain.vKey("label", "app"))))
|
||||
.isPresent();
|
||||
}
|
||||
|
||||
@Test
|
||||
void applyLabelDiffs_newLabel() {
|
||||
persistActiveDomain("label.app");
|
||||
ReservedListDao.save(
|
||||
new ReservedList.Builder()
|
||||
.setReservedListMap(
|
||||
ImmutableMap.of(
|
||||
"label",
|
||||
ReservedListEntry.create(
|
||||
"label", ReservationType.RESERVED_FOR_SPECIFIC_USE, null)))
|
||||
.setName("page_reserved")
|
||||
.setCreationTimestamp(fakeClock.nowUtc())
|
||||
.build());
|
||||
ReservedList reservedList = ReservedList.get("page_reserved").get();
|
||||
tm().transact(() -> tm().put(page.asBuilder().setReservedLists(reservedList).build()));
|
||||
|
||||
when(idnChecker.getForbiddingTlds(any()))
|
||||
.thenReturn(Sets.difference(ImmutableSet.of(dev), ImmutableSet.of()).immutableCopy());
|
||||
when(idnChecker.getSupportingTlds(any())).thenReturn(ImmutableSet.of(app, page));
|
||||
when(schedule.jobCreationTime()).thenReturn(fakeClock.nowUtc());
|
||||
|
||||
ImmutableList<UnblockableDomain> unblockableDomains =
|
||||
applyLabelDiff(
|
||||
ImmutableList.of(BlockLabel.of("label", LabelType.CREATE, ImmutableSet.of())),
|
||||
idnChecker,
|
||||
schedule,
|
||||
fakeClock.nowUtc());
|
||||
assertThat(unblockableDomains)
|
||||
.containsExactly(
|
||||
UnblockableDomain.of("label.app", UnblockableDomain.Reason.REGISTERED),
|
||||
UnblockableDomain.of("label.page", UnblockableDomain.Reason.RESERVED),
|
||||
UnblockableDomain.of("label.dev", UnblockableDomain.Reason.INVALID));
|
||||
assertThat(tm().transact(() -> tm().loadByKeyIfPresent(BsaLabel.vKey("label")))).isPresent();
|
||||
assertThat(
|
||||
tm().transact(() -> tm().loadByKey(BsaUnblockableDomain.vKey("label", "app")).reason))
|
||||
.isEqualTo(Reason.REGISTERED);
|
||||
assertThat(
|
||||
tm().transact(() -> tm().loadByKey(BsaUnblockableDomain.vKey("label", "page")).reason))
|
||||
.isEqualTo(Reason.RESERVED);
|
||||
assertThat(
|
||||
tm().transact(() -> tm().loadByKeyIfPresent(BsaUnblockableDomain.vKey("label", "dev"))))
|
||||
.isEmpty();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,205 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static google.registry.bsa.persistence.Queries.deleteBsaLabelByLabels;
|
||||
import static google.registry.bsa.persistence.Queries.queryBsaLabelByLabels;
|
||||
import static google.registry.bsa.persistence.Queries.queryBsaUnblockableDomainByLabels;
|
||||
import static google.registry.bsa.persistence.Queries.queryLivesDomains;
|
||||
import static google.registry.bsa.persistence.Queries.queryUnblockablesByNames;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
import static google.registry.testing.DatabaseHelper.createTlds;
|
||||
import static google.registry.testing.DatabaseHelper.newDomain;
|
||||
import static google.registry.testing.DatabaseHelper.persistDomainAsDeleted;
|
||||
import static google.registry.testing.DatabaseHelper.persistNewRegistrar;
|
||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.bsa.persistence.BsaUnblockableDomain.Reason;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationWithCoverageExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link Queries}. */
|
||||
class QueriesTest {
|
||||
|
||||
FakeClock fakeClock = new FakeClock(DateTime.parse("2023-11-09T02:08:57.880Z"));
|
||||
|
||||
@RegisterExtension
|
||||
final JpaIntegrationWithCoverageExtension jpa =
|
||||
new JpaTestExtensions.Builder().withClock(fakeClock).buildIntegrationWithCoverageExtension();
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
tm().transact(
|
||||
() -> {
|
||||
tm().putAll(
|
||||
ImmutableList.of(
|
||||
new BsaLabel("label1", fakeClock.nowUtc()),
|
||||
new BsaLabel("label2", fakeClock.nowUtc()),
|
||||
new BsaLabel("label3", fakeClock.nowUtc())));
|
||||
tm().putAll(
|
||||
ImmutableList.of(
|
||||
BsaUnblockableDomain.of("label1.app", Reason.REGISTERED),
|
||||
BsaUnblockableDomain.of("label1.dev", Reason.RESERVED),
|
||||
BsaUnblockableDomain.of("label2.page", Reason.REGISTERED),
|
||||
BsaUnblockableDomain.of("label3.app", Reason.REGISTERED)));
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
void queryBsaUnblockableDomainByLabels_oneLabel() {
|
||||
assertThat(
|
||||
tm().transact(
|
||||
() ->
|
||||
queryBsaUnblockableDomainByLabels(ImmutableList.of("label1"))
|
||||
.map(BsaUnblockableDomain::toVkey)
|
||||
.collect(toImmutableList())))
|
||||
.containsExactly(
|
||||
BsaUnblockableDomain.vKey("label1", "app"), BsaUnblockableDomain.vKey("label1", "dev"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void queryBsaUnblockableDomainByLabels_twoLabels() {
|
||||
assertThat(
|
||||
tm().transact(
|
||||
() ->
|
||||
queryBsaUnblockableDomainByLabels(ImmutableList.of("label1", "label2"))
|
||||
.map(BsaUnblockableDomain::toVkey)
|
||||
.collect(toImmutableList())))
|
||||
.containsExactly(
|
||||
BsaUnblockableDomain.vKey("label1", "app"),
|
||||
BsaUnblockableDomain.vKey("label1", "dev"),
|
||||
BsaUnblockableDomain.vKey("label2", "page"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void queryBsaLabelByLabels_oneLabel() {
|
||||
assertThat(
|
||||
tm().transact(
|
||||
() ->
|
||||
queryBsaLabelByLabels(ImmutableList.of("label1"))
|
||||
.collect(toImmutableList())))
|
||||
.containsExactly(new BsaLabel("label1", fakeClock.nowUtc()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void queryBsaLabelByLabels_twoLabels() {
|
||||
assertThat(
|
||||
tm().transact(
|
||||
() ->
|
||||
queryBsaLabelByLabels(ImmutableList.of("label1", "label2"))
|
||||
.collect(toImmutableList())))
|
||||
.containsExactly(
|
||||
new BsaLabel("label1", fakeClock.nowUtc()), new BsaLabel("label2", fakeClock.nowUtc()));
|
||||
}
|
||||
|
||||
@Test
|
||||
void deleteBsaLabelByLabels_oneLabel() {
|
||||
assertThat(tm().transact(() -> deleteBsaLabelByLabels(ImmutableList.of("label1"))))
|
||||
.isEqualTo(1);
|
||||
assertThat(tm().transact(() -> tm().loadAllOf(BsaLabel.class)))
|
||||
.containsExactly(
|
||||
new BsaLabel("label2", fakeClock.nowUtc()), new BsaLabel("label3", fakeClock.nowUtc()));
|
||||
assertThat(
|
||||
tm().transact(
|
||||
() ->
|
||||
tm().loadAllOfStream(BsaUnblockableDomain.class)
|
||||
.map(BsaUnblockableDomain::toVkey)
|
||||
.collect(toImmutableList())))
|
||||
.containsExactly(
|
||||
BsaUnblockableDomain.vKey("label2", "page"),
|
||||
BsaUnblockableDomain.vKey("label3", "app"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void deleteBsaLabelByLabels_twoLabels() {
|
||||
assertThat(tm().transact(() -> deleteBsaLabelByLabels(ImmutableList.of("label1", "label2"))))
|
||||
.isEqualTo(2);
|
||||
assertThat(tm().transact(() -> tm().loadAllOf(BsaLabel.class)))
|
||||
.containsExactly(new BsaLabel("label3", fakeClock.nowUtc()));
|
||||
assertThat(
|
||||
tm().transact(
|
||||
() ->
|
||||
tm().loadAllOfStream(BsaUnblockableDomain.class)
|
||||
.map(BsaUnblockableDomain::toVkey)
|
||||
.collect(toImmutableList())))
|
||||
.containsExactly(BsaUnblockableDomain.vKey("label3", "app"));
|
||||
}
|
||||
|
||||
private void setupUnblockableDomains() {
|
||||
tm().transact(
|
||||
() ->
|
||||
tm().insertAll(
|
||||
ImmutableList.of(
|
||||
new BsaLabel("a", fakeClock.nowUtc()),
|
||||
new BsaLabel("b", fakeClock.nowUtc()))));
|
||||
BsaUnblockableDomain a1 = new BsaUnblockableDomain("a", "tld1", Reason.RESERVED);
|
||||
BsaUnblockableDomain b1 = new BsaUnblockableDomain("b", "tld1", Reason.REGISTERED);
|
||||
BsaUnblockableDomain a2 = new BsaUnblockableDomain("a", "tld2", Reason.REGISTERED);
|
||||
tm().transact(() -> tm().insertAll(ImmutableList.of(a1, b1, a2)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void queryUnblockablesByNames_singleName_found() {
|
||||
setupUnblockableDomains();
|
||||
assertThat(tm().transact(() -> queryUnblockablesByNames(ImmutableSet.of("a.tld1"))))
|
||||
.containsExactly("a.tld1");
|
||||
}
|
||||
|
||||
@Test
|
||||
void queryUnblockablesByNames_singleName_notFound() {
|
||||
setupUnblockableDomains();
|
||||
assertThat(tm().transact(() -> queryUnblockablesByNames(ImmutableSet.of("c.tld3")))).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void queryUnblockablesByNames_multipleNames() {
|
||||
setupUnblockableDomains();
|
||||
assertThat(
|
||||
tm().transact(
|
||||
() -> queryUnblockablesByNames(ImmutableSet.of("a.tld1", "b.tld1", "c.tld3"))))
|
||||
.containsExactly("a.tld1", "b.tld1");
|
||||
}
|
||||
|
||||
@Test
|
||||
void queryLivesDomains_onlyLiveDomainsReturned() {
|
||||
DateTime testStartTime = fakeClock.nowUtc();
|
||||
createTlds("tld");
|
||||
persistNewRegistrar("TheRegistrar");
|
||||
// time 0:
|
||||
persistResource(
|
||||
newDomain("d1.tld").asBuilder().setCreationTimeForTest(fakeClock.nowUtc()).build());
|
||||
// time 0, deletion time 1
|
||||
persistDomainAsDeleted(
|
||||
newDomain("will-delete.tld").asBuilder().setCreationTimeForTest(fakeClock.nowUtc()).build(),
|
||||
fakeClock.nowUtc().plusMillis(1));
|
||||
fakeClock.advanceOneMilli();
|
||||
// time 1
|
||||
persistResource(
|
||||
newDomain("d2.tld").asBuilder().setCreationTimeForTest(fakeClock.nowUtc()).build());
|
||||
fakeClock.advanceOneMilli();
|
||||
// Now is time 2
|
||||
assertThat(tm().transact(() -> queryLivesDomains(testStartTime, fakeClock.nowUtc())))
|
||||
.containsExactly("d1.tld", "d2.tld");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,140 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.bsa.persistence;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.bsa.RefreshStage.APPLY_CHANGES;
|
||||
import static google.registry.bsa.RefreshStage.DONE;
|
||||
import static google.registry.persistence.transaction.TransactionManagerFactory.tm;
|
||||
|
||||
import google.registry.bsa.DownloadStage;
|
||||
import google.registry.bsa.RefreshStage;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationWithCoverageExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
import java.util.Optional;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link RefreshScheduler}. */
|
||||
public class RefreshSchedulerTest {
|
||||
|
||||
FakeClock fakeClock = new FakeClock(DateTime.parse("2023-11-09T02:08:57.880Z"));
|
||||
|
||||
@RegisterExtension
|
||||
final JpaIntegrationWithCoverageExtension jpa =
|
||||
new JpaTestExtensions.Builder().withClock(fakeClock).buildIntegrationWithCoverageExtension();
|
||||
|
||||
RefreshScheduler scheduler;
|
||||
|
||||
@BeforeEach
|
||||
void setup() {
|
||||
scheduler = new RefreshScheduler();
|
||||
}
|
||||
|
||||
@Test
|
||||
void schedule_noPrevRefresh_noPrevDownload() {
|
||||
Optional<RefreshSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void schedule_noPrevRefresh_withOngoingPrevDownload() {
|
||||
tm().transact(() -> tm().insert(new BsaDownload()));
|
||||
Optional<RefreshSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void schedule_NoPreviousRefresh_withCompletedPrevDownload() {
|
||||
tm().transact(() -> tm().insert(new BsaDownload().setStage(DownloadStage.DONE)));
|
||||
DateTime downloadTime = fakeClock.nowUtc();
|
||||
fakeClock.advanceOneMilli();
|
||||
|
||||
Optional<RefreshSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isPresent();
|
||||
RefreshSchedule schedule = scheduleOptional.get();
|
||||
|
||||
assertThat(schedule.jobCreationTime()).isEqualTo(fakeClock.nowUtc());
|
||||
assertThat(schedule.stage()).isEqualTo(RefreshStage.CHECK_FOR_CHANGES);
|
||||
assertThat(schedule.prevRefreshTime()).isEqualTo(downloadTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
void schedule_firstRefreshOngoing() {
|
||||
tm().transact(() -> tm().insert(new BsaDownload().setStage(DownloadStage.DONE)));
|
||||
DateTime downloadTime = fakeClock.nowUtc();
|
||||
fakeClock.advanceOneMilli();
|
||||
|
||||
tm().transact(() -> tm().insert(new BsaDomainRefresh().setStage(APPLY_CHANGES)));
|
||||
DateTime refreshStartTime = fakeClock.nowUtc();
|
||||
fakeClock.advanceOneMilli();
|
||||
|
||||
Optional<RefreshSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isPresent();
|
||||
RefreshSchedule schedule = scheduleOptional.get();
|
||||
|
||||
assertThat(schedule.jobCreationTime()).isEqualTo(refreshStartTime);
|
||||
assertThat(schedule.stage()).isEqualTo(APPLY_CHANGES);
|
||||
assertThat(schedule.prevRefreshTime()).isEqualTo(downloadTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
void schedule_firstRefreshDone() {
|
||||
tm().transact(() -> tm().insert(new BsaDomainRefresh().setStage(DONE)));
|
||||
DateTime prevRefreshStartTime = fakeClock.nowUtc();
|
||||
fakeClock.advanceOneMilli();
|
||||
|
||||
Optional<RefreshSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isPresent();
|
||||
RefreshSchedule schedule = scheduleOptional.get();
|
||||
|
||||
assertThat(schedule.jobCreationTime()).isEqualTo(fakeClock.nowUtc());
|
||||
assertThat(schedule.stage()).isEqualTo(RefreshStage.CHECK_FOR_CHANGES);
|
||||
assertThat(schedule.prevRefreshTime()).isEqualTo(prevRefreshStartTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
void schedule_ongoingRefreshWithPrevCompletion() {
|
||||
tm().transact(() -> tm().insert(new BsaDomainRefresh().setStage(DONE)));
|
||||
DateTime prevRefreshStartTime = fakeClock.nowUtc();
|
||||
fakeClock.advanceOneMilli();
|
||||
tm().transact(() -> tm().insert(new BsaDomainRefresh().setStage(APPLY_CHANGES)));
|
||||
DateTime ongoingRefreshStartTime = fakeClock.nowUtc();
|
||||
fakeClock.advanceOneMilli();
|
||||
|
||||
Optional<RefreshSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isPresent();
|
||||
RefreshSchedule schedule = scheduleOptional.get();
|
||||
|
||||
assertThat(schedule.jobCreationTime()).isEqualTo(ongoingRefreshStartTime);
|
||||
assertThat(schedule.stage()).isEqualTo(APPLY_CHANGES);
|
||||
assertThat(schedule.prevRefreshTime()).isEqualTo(prevRefreshStartTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
void schedule_blockedByOngoingDownload() {
|
||||
tm().transact(() -> tm().insert(new BsaDomainRefresh().setStage(DONE)));
|
||||
fakeClock.advanceOneMilli();
|
||||
tm().transact(() -> tm().insert(new BsaDownload()));
|
||||
fakeClock.advanceOneMilli();
|
||||
|
||||
Optional<RefreshSchedule> scheduleOptional = scheduler.schedule();
|
||||
assertThat(scheduleOptional).isEmpty();
|
||||
}
|
||||
}
|
|
@ -16,6 +16,7 @@ package google.registry.model.tld;
|
|||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
import static com.google.common.truth.Truth8.assertThat;
|
||||
import static google.registry.model.tld.Tlds.hasActiveBsaEnrollment;
|
||||
import static google.registry.testing.DatabaseHelper.createTlds;
|
||||
import static google.registry.testing.DatabaseHelper.newTld;
|
||||
import static google.registry.testing.DatabaseHelper.persistResource;
|
||||
|
@ -25,15 +26,20 @@ import com.google.common.net.InternetDomainName;
|
|||
import google.registry.model.tld.Tld.TldType;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions;
|
||||
import google.registry.persistence.transaction.JpaTestExtensions.JpaIntegrationTestExtension;
|
||||
import google.registry.testing.FakeClock;
|
||||
import google.registry.util.Clock;
|
||||
import java.util.Optional;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.RegisterExtension;
|
||||
|
||||
/** Unit tests for {@link Tlds}. */
|
||||
class TldsTest {
|
||||
|
||||
Clock fakeClock = new FakeClock();
|
||||
|
||||
@RegisterExtension
|
||||
final JpaIntegrationTestExtension jpa =
|
||||
new JpaTestExtensions.Builder().buildIntegrationTestExtension();
|
||||
new JpaTestExtensions.Builder().withClock(fakeClock).buildIntegrationTestExtension();
|
||||
|
||||
private void initTestTlds() {
|
||||
createTlds("foo", "a.b.c"); // Test a multipart tld.
|
||||
|
@ -89,4 +95,44 @@ class TldsTest {
|
|||
// Substring tld matches aren't considered.
|
||||
assertThat(Tlds.findTldForName(InternetDomainName.from("example.barfoo"))).isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testHasActiveBsaEnrollment_noneEnrolled() {
|
||||
initTestTlds();
|
||||
assertThat(hasActiveBsaEnrollment(fakeClock.nowUtc())).isFalse();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testHasActiveBsaEnrollment_enrolledInTheFuture() {
|
||||
initTestTlds();
|
||||
persistResource(
|
||||
Tld.get("foo")
|
||||
.asBuilder()
|
||||
.setBsaEnrollStartTime(Optional.of(fakeClock.nowUtc().plusSeconds(1)))
|
||||
.build());
|
||||
assertThat(hasActiveBsaEnrollment(fakeClock.nowUtc())).isFalse();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testHasActiveBsaEnrollment_enrolledIsTestTld() {
|
||||
initTestTlds();
|
||||
persistResource(
|
||||
Tld.get("foo")
|
||||
.asBuilder()
|
||||
.setTldType(TldType.TEST)
|
||||
.setBsaEnrollStartTime(Optional.of(fakeClock.nowUtc().minus(1)))
|
||||
.build());
|
||||
assertThat(hasActiveBsaEnrollment(fakeClock.nowUtc())).isFalse();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testHasActiveBsaEnrollment_enrolled() {
|
||||
initTestTlds();
|
||||
persistResource(
|
||||
Tld.get("foo")
|
||||
.asBuilder()
|
||||
.setBsaEnrollStartTime(Optional.of(fakeClock.nowUtc().minus(1)))
|
||||
.build());
|
||||
assertThat(hasActiveBsaEnrollment(fakeClock.nowUtc())).isTrue();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,41 @@
|
|||
// Copyright 2023 The Nomulus Authors. All Rights Reserved.
|
||||
//
|
||||
// Licensed 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 google.registry.module.bsa;
|
||||
|
||||
import static com.google.common.truth.Truth.assertThat;
|
||||
|
||||
import google.registry.request.Action.Service;
|
||||
import google.registry.request.RouterDisplayHelper;
|
||||
import google.registry.testing.GoldenFileTestHelper;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
/** Unit tests for {@link BsaRequestComponent}. */
|
||||
public class BsaRequestComponentTest {
|
||||
|
||||
@Test
|
||||
void testRoutingMap() {
|
||||
GoldenFileTestHelper.assertThatRoutesFromComponent(BsaRequestComponent.class)
|
||||
.describedAs("bsa routing map")
|
||||
.isEqualToGolden(BsaRequestComponent.class, "bsa_routing.txt");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRoutingService() {
|
||||
assertThat(
|
||||
RouterDisplayHelper.extractHumanReadableRoutesWithWrongService(
|
||||
BsaRequestComponent.class, Service.BSA))
|
||||
.isEmpty();
|
||||
}
|
||||
}
|
|
@ -16,10 +16,10 @@ package google.registry.schema.integration;
|
|||
|
||||
import static com.google.common.truth.Truth.assert_;
|
||||
|
||||
import google.registry.bsa.persistence.BsaDomainInUseTest;
|
||||
import google.registry.bsa.persistence.BsaDomainRefreshTest;
|
||||
import google.registry.bsa.persistence.BsaDownloadTest;
|
||||
import google.registry.bsa.persistence.BsaLabelTest;
|
||||
import google.registry.bsa.persistence.BsaUnblockableDomainTest;
|
||||
import google.registry.model.billing.BillingBaseTest;
|
||||
import google.registry.model.common.CursorTest;
|
||||
import google.registry.model.common.DnsRefreshRequestTest;
|
||||
|
@ -86,10 +86,10 @@ import org.junit.runner.RunWith;
|
|||
BeforeSuiteTest.class,
|
||||
AllocationTokenTest.class,
|
||||
BillingBaseTest.class,
|
||||
BsaDomainInUseTest.class,
|
||||
BsaDomainRefreshTest.class,
|
||||
BsaDownloadTest.class,
|
||||
BsaLabelTest.class,
|
||||
BsaUnblockableDomainTest.class,
|
||||
BulkPricingPackageTest.class,
|
||||
ClaimsListDaoTest.class,
|
||||
ContactHistoryTest.class,
|
||||
|
|
|
@ -0,0 +1,3 @@
|
|||
PATH CLASS METHODS OK AUTH_METHODS MIN USER_POLICY
|
||||
/_dr/task/bsaDownload BsaDownloadAction GET,POST n API APP ADMIN
|
||||
/_dr/task/bsaRefresh BsaRefreshAction GET,POST n API APP ADMIN
|
|
@ -3678,7 +3678,7 @@ td.section {
|
|||
timestamptz not null
|
||||
</text>
|
||||
<text text-anchor="start" x="4321" y="-5351.3" font-family="Helvetica,sans-Serif" font-size="14.00">
|
||||
stage
|
||||
refreshStage
|
||||
</text>
|
||||
<text text-anchor="start" x="4462" y="-5351.3" font-family="Helvetica,sans-Serif" font-size="14.00">
|
||||
</text>
|
||||
|
@ -3735,7 +3735,7 @@ td.section {
|
|||
timestamptz not null
|
||||
</text>
|
||||
<text text-anchor="start" x="4335" y="-5493.3" font-family="Helvetica,sans-Serif" font-size="14.00">
|
||||
stage
|
||||
refreshStage
|
||||
</text>
|
||||
<text text-anchor="start" x="4471" y="-5493.3" font-family="Helvetica,sans-Serif" font-size="14.00">
|
||||
</text>
|
||||
|
@ -8120,7 +8120,7 @@ td.section {
|
|||
</tr>
|
||||
<tr>
|
||||
<td class="spacer"></td>
|
||||
<td class="minwidth">stage</td>
|
||||
<td class="minwidth">refreshStage</td>
|
||||
<td class="minwidth">text not null</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -8194,7 +8194,7 @@ td.section {
|
|||
</tr>
|
||||
<tr>
|
||||
<td class="spacer"></td>
|
||||
<td class="minwidth">stage</td>
|
||||
<td class="minwidth">refreshStage</td>
|
||||
<td class="minwidth">text not null</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
|
|
@ -85,14 +85,6 @@
|
|||
primary key (billing_recurrence_id)
|
||||
);
|
||||
|
||||
create table "BsaDomainInUse" (
|
||||
label text not null,
|
||||
tld text not null,
|
||||
creation_time timestamptz not null,
|
||||
reason text not null,
|
||||
primary key (label, tld)
|
||||
);
|
||||
|
||||
create table "BsaDomainRefresh" (
|
||||
job_id bigserial not null,
|
||||
creation_time timestamptz not null,
|
||||
|
@ -116,6 +108,14 @@
|
|||
primary key (label)
|
||||
);
|
||||
|
||||
create table "BsaUnblockableDomain" (
|
||||
label text not null,
|
||||
tld text not null,
|
||||
creation_time timestamptz not null,
|
||||
reason text not null,
|
||||
primary key (label, tld)
|
||||
);
|
||||
|
||||
create table "ClaimsEntry" (
|
||||
revision_id int8 not null,
|
||||
domain_label text not null,
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue