From 4bc216c85f9712da398093acad644632a589a608 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Wed, 4 Jul 2018 16:08:56 +0200 Subject: [PATCH 01/26] ARUHA-1757: Added log compaction functionality; --- .../repository/kafka/KafkaRepositoryAT.java | 14 ++--- .../org/zalando/nakadi/domain/BatchItem.java | 10 ++++ .../zalando/nakadi/domain/CleanupPolicy.java | 5 ++ .../zalando/nakadi/domain/EventTypeBase.java | 41 ++++++++++--- .../partitioning/HashPartitionStrategy.java | 2 +- .../nakadi/repository/TopicRepository.java | 16 ++--- .../kafka/KafkaTopicRepository.java | 54 ++++------------- .../nakadi/service/EventPublisher.java | 49 +++++++++------ .../nakadi/service/EventTypeService.java | 39 ++---------- .../service/timeline/TimelineService.java | 53 +++++++--------- src/main/resources/application.yml | 15 +++-- .../controller/EventTypeControllerTest.java | 60 +++++-------------- .../nakadi/service/EventTypeServiceTest.java | 14 ++--- .../service/timeline/TimelineServiceTest.java | 6 +- 14 files changed, 161 insertions(+), 217 deletions(-) create mode 100644 src/main/java/org/zalando/nakadi/domain/CleanupPolicy.java diff --git a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java index dae71b9e45..0aacf9afda 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java @@ -13,6 +13,7 @@ import org.zalando.nakadi.config.NakadiSettings; import org.zalando.nakadi.domain.BatchFactory; import org.zalando.nakadi.domain.BatchItem; +import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.EventPublishingStatus; import org.zalando.nakadi.repository.zookeeper.ZooKeeperHolder; import org.zalando.nakadi.repository.zookeeper.ZookeeperSettings; @@ -27,11 +28,7 @@ import static org.echocat.jomon.runtime.concurrent.Retryer.executeWithRetry; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.arrayWithSize; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasItem; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.*; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -49,6 +46,7 @@ public class KafkaRepositoryAT extends BaseAT { private static final int NAKADI_POLL_TIMEOUT = 10000; private static final Long RETENTION_TIME = 100L; private static final Long DEFAULT_TOPIC_RETENTION = 100000000L; + private static final CleanupPolicy CLEANUP_POLICY = CleanupPolicy.DELETE; private static final int KAFKA_REQUEST_TIMEOUT = 30000; private static final int KAFKA_BATCH_SIZE = 1048576; private static final long KAFKA_LINGER_MS = 0; @@ -95,7 +93,8 @@ public void setup() { @SuppressWarnings("unchecked") public void whenCreateTopicThenTopicIsCreated() throws Exception { // ACT // - final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, RETENTION_TIME); + final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, RETENTION_TIME, + CLEANUP_POLICY); // ASSERT // executeWithRetry(() -> { @@ -162,7 +161,8 @@ public void whenBulkSendSuccessfullyThenUpdateBatchItemStatus() throws Exception @SuppressWarnings("unchecked") public void whenCreateTopicWithRetentionTime() throws Exception { // ACT // - final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, RETENTION_TIME); + final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, RETENTION_TIME, + CLEANUP_POLICY); // ASSERT // executeWithRetry(() -> Assert.assertEquals( diff --git a/src/main/java/org/zalando/nakadi/domain/BatchItem.java b/src/main/java/org/zalando/nakadi/domain/BatchItem.java index 2a8a158492..1e309c9e15 100644 --- a/src/main/java/org/zalando/nakadi/domain/BatchItem.java +++ b/src/main/java/org/zalando/nakadi/domain/BatchItem.java @@ -61,6 +61,7 @@ public static EmptyInjectionConfiguration build(final int position, final boolea private final List skipCharacters; private String partition; private String brokerId; + private String eventKey; private int eventSize; public BatchItem( @@ -101,6 +102,15 @@ public String getPartition() { return partition; } + @Nullable + public String getEventKey() { + return eventKey; + } + + public void setEventKey(@Nullable final String eventKey) { + this.eventKey = eventKey; + } + @Nullable public String getBrokerId() { return brokerId; diff --git a/src/main/java/org/zalando/nakadi/domain/CleanupPolicy.java b/src/main/java/org/zalando/nakadi/domain/CleanupPolicy.java new file mode 100644 index 0000000000..a119b26021 --- /dev/null +++ b/src/main/java/org/zalando/nakadi/domain/CleanupPolicy.java @@ -0,0 +1,5 @@ +package org.zalando.nakadi.domain; + +public enum CleanupPolicy { + DELETE, COMPACT +} diff --git a/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java b/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java index df4862707f..5d99e09856 100644 --- a/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java +++ b/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java @@ -19,6 +19,7 @@ public class EventTypeBase { private static final List EMPTY_PARTITION_KEY_FIELDS = ImmutableList.of(); private static final List EMPTY_ORDERING_KEY_FIELDS = ImmutableList.of(); + private static final List EMPTY_PARTITION_COMPACTION_KEYS = ImmutableList.of(); @NotNull @Pattern(regexp = "[a-zA-Z][-0-9a-zA-Z_]*(\\.[0-9a-zA-Z][-0-9a-zA-Z_]*)*", message = "format not allowed") @@ -42,6 +43,12 @@ public class EventTypeBase { @Nullable private List partitionKeyFields; + @NotNull + private CleanupPolicy cleanupPolicy; + + @Nullable + private List partitionCompactionKeys; + @Nullable private List orderingKeyFields; @@ -72,17 +79,18 @@ public EventTypeBase() { this.partitionStrategy = PartitionStrategy.RANDOM_STRATEGY; this.options = new EventTypeOptions(); this.compatibilityMode = CompatibilityMode.FORWARD; + this.cleanupPolicy = CleanupPolicy.DELETE; } public EventTypeBase(final String name, final String owningApplication, - final EventCategory category, - final List validationStrategies, - final List enrichmentStrategies, - final String partitionStrategy, - final List partitionKeyFields, final EventTypeSchemaBase schema, - final EventTypeStatistics defaultStatistic, - final EventTypeOptions options, - final CompatibilityMode compatibilityMode) { + final EventCategory category, + final List validationStrategies, + final List enrichmentStrategies, + final String partitionStrategy, + final List partitionKeyFields, final EventTypeSchemaBase schema, + final EventTypeStatistics defaultStatistic, + final EventTypeOptions options, + final CompatibilityMode compatibilityMode) { this.name = name; this.owningApplication = owningApplication; this.category = category; @@ -173,6 +181,23 @@ public void setPartitionKeyFields(final List partitionKeyFields) { this.partitionKeyFields = partitionKeyFields; } + public CleanupPolicy getCleanupPolicy() { + return cleanupPolicy; + } + + public void setCleanupPolicy(CleanupPolicy cleanupPolicy) { + this.cleanupPolicy = cleanupPolicy; + } + + public List getPartitionCompactionKeys() { + return unmodifiableList(partitionCompactionKeys != null ? partitionCompactionKeys : + EMPTY_PARTITION_COMPACTION_KEYS); + } + + public void setPartitionCompactionKeys(@Nullable List partitionCompactionKeys) { + this.partitionCompactionKeys = partitionCompactionKeys; + } + public List getOrderingKeyFields() { return unmodifiableList(orderingKeyFields != null ? orderingKeyFields : EMPTY_ORDERING_KEY_FIELDS); } diff --git a/src/main/java/org/zalando/nakadi/partitioning/HashPartitionStrategy.java b/src/main/java/org/zalando/nakadi/partitioning/HashPartitionStrategy.java index 3e6f782514..0fd0337a4f 100644 --- a/src/main/java/org/zalando/nakadi/partitioning/HashPartitionStrategy.java +++ b/src/main/java/org/zalando/nakadi/partitioning/HashPartitionStrategy.java @@ -19,7 +19,7 @@ @Component public class HashPartitionStrategy implements PartitionStrategy { - private static final String DATA_PATH_PREFIX = JsonSchemaEnrichment.DATA_CHANGE_WRAP_FIELD + "."; + public static final String DATA_PATH_PREFIX = JsonSchemaEnrichment.DATA_CHANGE_WRAP_FIELD + "."; private final HashPartitionStrategyCrutch hashPartitioningCrutch; private final StringHash stringHash; diff --git a/src/main/java/org/zalando/nakadi/repository/TopicRepository.java b/src/main/java/org/zalando/nakadi/repository/TopicRepository.java index 0a6b0563fb..aeee847b32 100644 --- a/src/main/java/org/zalando/nakadi/repository/TopicRepository.java +++ b/src/main/java/org/zalando/nakadi/repository/TopicRepository.java @@ -1,18 +1,9 @@ package org.zalando.nakadi.repository; -import org.zalando.nakadi.domain.BatchItem; -import org.zalando.nakadi.domain.NakadiCursor; -import org.zalando.nakadi.domain.PartitionEndStatistics; -import org.zalando.nakadi.domain.PartitionStatistics; -import org.zalando.nakadi.domain.Timeline; +import org.zalando.nakadi.domain.*; import org.zalando.nakadi.exceptions.InvalidCursorException; import org.zalando.nakadi.exceptions.NakadiException; -import org.zalando.nakadi.exceptions.runtime.EventPublishingException; -import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; -import org.zalando.nakadi.exceptions.runtime.TopicConfigException; -import org.zalando.nakadi.exceptions.runtime.TopicCreationException; -import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; -import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; +import org.zalando.nakadi.exceptions.runtime.*; import java.util.Collection; import java.util.List; @@ -38,7 +29,8 @@ public String getPartition() { } } - String createTopic(int partitionCount, Long retentionTimeMs) throws TopicCreationException; + String createTopic(int partitionCount, Long retentionTimeMs, CleanupPolicy cleanupPolicy) + throws TopicCreationException; void deleteTopic(String topic) throws TopicDeletionException; diff --git a/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java index 9b9632a51d..f37a911bb0 100644 --- a/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java +++ b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java @@ -10,31 +10,15 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.InterruptException; -import org.apache.kafka.common.errors.NetworkException; -import org.apache.kafka.common.errors.NotLeaderForPartitionException; -import org.apache.kafka.common.errors.TopicExistsException; -import org.apache.kafka.common.errors.UnknownServerException; -import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.errors.*; import org.echocat.jomon.runtime.concurrent.RetryForSpecifiedTimeStrategy; import org.echocat.jomon.runtime.concurrent.Retryer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.zalando.nakadi.config.NakadiSettings; -import org.zalando.nakadi.domain.BatchItem; -import org.zalando.nakadi.domain.EventPublishingStatus; -import org.zalando.nakadi.domain.EventPublishingStep; -import org.zalando.nakadi.domain.NakadiCursor; -import org.zalando.nakadi.domain.PartitionEndStatistics; -import org.zalando.nakadi.domain.PartitionStatistics; -import org.zalando.nakadi.domain.Timeline; +import org.zalando.nakadi.domain.*; import org.zalando.nakadi.exceptions.InvalidCursorException; -import org.zalando.nakadi.exceptions.runtime.EventPublishingException; -import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; -import org.zalando.nakadi.exceptions.runtime.TopicConfigException; -import org.zalando.nakadi.exceptions.runtime.TopicCreationException; -import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; -import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; +import org.zalando.nakadi.exceptions.runtime.*; import org.zalando.nakadi.repository.EventConsumer; import org.zalando.nakadi.repository.TopicRepository; import org.zalando.nakadi.repository.zookeeper.ZooKeeperHolder; @@ -42,21 +26,8 @@ import org.zalando.nakadi.util.UUIDGenerator; import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; +import java.util.*; +import java.util.concurrent.*; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -65,10 +36,7 @@ import static com.google.common.collect.Lists.newArrayList; import static java.util.Collections.unmodifiableList; import static java.util.stream.Collectors.toList; -import static org.zalando.nakadi.domain.CursorError.NULL_OFFSET; -import static org.zalando.nakadi.domain.CursorError.NULL_PARTITION; -import static org.zalando.nakadi.domain.CursorError.PARTITION_NOT_FOUND; -import static org.zalando.nakadi.domain.CursorError.UNAVAILABLE; +import static org.zalando.nakadi.domain.CursorError.*; public class KafkaTopicRepository implements TopicRepository { @@ -108,7 +76,7 @@ public List listTopics() throws TopicRepositoryException { } @Override - public String createTopic(final int partitionCount, final Long retentionTimeMs) + public String createTopic(final int partitionCount, final Long retentionTimeMs, final CleanupPolicy cleanupPolicy) throws TopicCreationException { if (retentionTimeMs == null) { throw new IllegalArgumentException("Retention time can not be null"); @@ -118,18 +86,20 @@ public String createTopic(final int partitionCount, final Long retentionTimeMs) partitionCount, nakadiSettings.getDefaultTopicReplicaFactor(), retentionTimeMs, - nakadiSettings.getDefaultTopicRotationMs()); + nakadiSettings.getDefaultTopicRotationMs(), + cleanupPolicy == CleanupPolicy.COMPACT ? "compact" : "delete"); return topicName; } private void createTopic(final String topic, final int partitionsNum, final int replicaFactor, - final long retentionMs, final long rotationMs) + final long retentionMs, final long rotationMs, final String cleanupPolicy) throws TopicCreationException { try { doWithZkUtils(zkUtils -> { final Properties topicConfig = new Properties(); topicConfig.setProperty("retention.ms", Long.toString(retentionMs)); topicConfig.setProperty("segment.ms", Long.toString(rotationMs)); + topicConfig.setProperty("cleanup.policy", cleanupPolicy); AdminUtils.createTopic(zkUtils, topic, partitionsNum, replicaFactor, topicConfig, RackAwareMode.Safe$.MODULE$); }); @@ -184,7 +154,7 @@ private static CompletableFuture publishItem( final ProducerRecord kafkaRecord = new ProducerRecord<>( topicId, KafkaCursor.toKafkaPartition(item.getPartition()), - item.getPartition(), + item.getEventKey(), item.dumpEventToString()); circuitBreaker.markStart(); diff --git a/src/main/java/org/zalando/nakadi/service/EventPublisher.java b/src/main/java/org/zalando/nakadi/service/EventPublisher.java index 424f12786c..33fc9d224a 100644 --- a/src/main/java/org/zalando/nakadi/service/EventPublisher.java +++ b/src/main/java/org/zalando/nakadi/service/EventPublisher.java @@ -1,33 +1,21 @@ package org.zalando.nakadi.service; +import org.json.JSONArray; import org.json.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; import org.zalando.nakadi.config.NakadiSettings; -import org.zalando.nakadi.domain.BatchFactory; -import org.zalando.nakadi.domain.BatchItem; -import org.zalando.nakadi.domain.BatchItemResponse; -import org.zalando.nakadi.domain.EventPublishResult; -import org.zalando.nakadi.domain.EventPublishingStatus; -import org.zalando.nakadi.domain.EventPublishingStep; -import org.zalando.nakadi.domain.EventType; -import org.zalando.nakadi.domain.Timeline; +import org.zalando.nakadi.domain.*; import org.zalando.nakadi.enrichment.Enrichment; -import org.zalando.nakadi.exceptions.EnrichmentException; -import org.zalando.nakadi.exceptions.InternalNakadiException; -import org.zalando.nakadi.exceptions.NoSuchEventTypeException; -import org.zalando.nakadi.exceptions.PartitioningException; -import org.zalando.nakadi.exceptions.runtime.AccessDeniedException; -import org.zalando.nakadi.exceptions.runtime.EventPublishingException; -import org.zalando.nakadi.exceptions.runtime.EventTypeTimeoutException; -import org.zalando.nakadi.exceptions.runtime.EventValidationException; -import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; +import org.zalando.nakadi.exceptions.*; +import org.zalando.nakadi.exceptions.runtime.*; import org.zalando.nakadi.partitioning.PartitionResolver; import org.zalando.nakadi.repository.db.EventTypeCache; import org.zalando.nakadi.service.timeline.TimelineService; import org.zalando.nakadi.service.timeline.TimelineSync; +import org.zalando.nakadi.util.JsonPathAccess; import org.zalando.nakadi.validation.EventTypeValidator; import org.zalando.nakadi.validation.ValidationError; @@ -38,6 +26,8 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; +import static org.zalando.nakadi.partitioning.HashPartitionStrategy.DATA_PATH_PREFIX; + @Component public class EventPublisher { @@ -99,6 +89,7 @@ EventPublishResult publishInternal(final String events, validate(batch, eventType); partition(batch, eventType); + compact(batch, eventType); enrich(batch, eventType); submit(batch, eventType); @@ -164,6 +155,30 @@ private void partition(final List batch, final EventType eventType) t } } + private void compact(final List batch, final EventType eventType) { + if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { + for (final BatchItem item : batch) { + final JsonPathAccess jsonPath = new JsonPathAccess(item.getEvent()); + final List compactionKeys = eventType.getPartitionCompactionKeys().stream() + .map(compactionKeyField -> EventCategory.DATA.equals(eventType.getCategory()) ? + DATA_PATH_PREFIX + compactionKeyField : compactionKeyField) + .map(compactionKeyField -> { + try { + return jsonPath.get(compactionKeyField).toString(); + } catch (final InvalidPartitionKeyFieldsException e) { + // this should be never thrown as we force users to make compaction keys to be required, + // so if compaction key is missing we should fail earlier on validation step + throw new InconsistentStateException( + "Unexpected exception occurred when assembling compaction key", e); + } + }) + .collect(Collectors.toList()); + final String compactionKeyStr = new JSONArray(compactionKeys).toString(); + item.setEventKey(compactionKeyStr); + } + } + } + private void validate(final List batch, final EventType eventType) throws EventValidationException, InternalNakadiException, NoSuchEventTypeException { for (final BatchItem item : batch) { diff --git a/src/main/java/org/zalando/nakadi/service/EventTypeService.java b/src/main/java/org/zalando/nakadi/service/EventTypeService.java index f113235929..8e4002f748 100644 --- a/src/main/java/org/zalando/nakadi/service/EventTypeService.java +++ b/src/main/java/org/zalando/nakadi/service/EventTypeService.java @@ -15,38 +15,10 @@ import org.springframework.transaction.TransactionException; import org.springframework.transaction.support.TransactionTemplate; import org.zalando.nakadi.config.NakadiSettings; -import org.zalando.nakadi.domain.CompatibilityMode; -import org.zalando.nakadi.domain.EventCategory; -import org.zalando.nakadi.domain.EventType; -import org.zalando.nakadi.domain.EventTypeBase; -import org.zalando.nakadi.domain.EventTypeOptions; -import org.zalando.nakadi.domain.EventTypeStatistics; -import org.zalando.nakadi.domain.Subscription; -import org.zalando.nakadi.domain.Timeline; +import org.zalando.nakadi.domain.*; import org.zalando.nakadi.enrichment.Enrichment; -import org.zalando.nakadi.exceptions.InternalNakadiException; -import org.zalando.nakadi.exceptions.NakadiException; -import org.zalando.nakadi.exceptions.NakadiRuntimeException; -import org.zalando.nakadi.exceptions.NoSuchEventTypeException; -import org.zalando.nakadi.exceptions.NoSuchPartitionStrategyException; -import org.zalando.nakadi.exceptions.NoSuchSubscriptionException; -import org.zalando.nakadi.exceptions.runtime.AccessDeniedException; -import org.zalando.nakadi.exceptions.runtime.ConflictException; -import org.zalando.nakadi.exceptions.runtime.DbWriteOperationsBlockedException; -import org.zalando.nakadi.exceptions.runtime.DuplicatedEventTypeNameException; -import org.zalando.nakadi.exceptions.runtime.EventTypeDeletionException; -import org.zalando.nakadi.exceptions.runtime.EventTypeOptionsValidationException; -import org.zalando.nakadi.exceptions.runtime.EventTypeUnavailableException; -import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; -import org.zalando.nakadi.exceptions.runtime.InvalidEventTypeException; -import org.zalando.nakadi.exceptions.runtime.NoEventTypeException; -import org.zalando.nakadi.exceptions.runtime.NotFoundException; -import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; -import org.zalando.nakadi.exceptions.runtime.TimelineException; -import org.zalando.nakadi.exceptions.runtime.TopicConfigException; -import org.zalando.nakadi.exceptions.runtime.TopicCreationException; -import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; -import org.zalando.nakadi.exceptions.runtime.UnableProcessException; +import org.zalando.nakadi.exceptions.*; +import org.zalando.nakadi.exceptions.runtime.*; import org.zalando.nakadi.partitioning.PartitionResolver; import org.zalando.nakadi.plugin.api.authz.AuthorizationService; import org.zalando.nakadi.repository.EventTypeRepository; @@ -155,9 +127,8 @@ public void create(final EventTypeBase eventType) eventTypeRepository.saveEventType(eventType); try { - timelineService.createDefaultTimeline(eventType.getName(), - partitionsCalculator.getBestPartitionsCount(eventType.getDefaultStatistic()), - eventType.getOptions().getRetentionTime()); + timelineService.createDefaultTimeline(eventType, + partitionsCalculator.getBestPartitionsCount(eventType.getDefaultStatistic())); } catch (final Exception e) { try { eventTypeRepository.removeEventType(eventType.getName()); diff --git a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java index 67c34b2ab1..81f43eb411 100644 --- a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java +++ b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java @@ -6,35 +6,17 @@ import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.beans.factory.annotation.Qualifier; +import org.springframework.beans.factory.annotation.Value; import org.springframework.stereotype.Service; import org.springframework.transaction.TransactionException; import org.springframework.transaction.support.TransactionTemplate; import org.zalando.nakadi.config.NakadiSettings; -import org.zalando.nakadi.domain.DefaultStorage; -import org.zalando.nakadi.domain.EventType; -import org.zalando.nakadi.domain.EventTypeBase; -import org.zalando.nakadi.domain.EventTypeResource; -import org.zalando.nakadi.domain.NakadiCursor; -import org.zalando.nakadi.domain.PartitionStatistics; -import org.zalando.nakadi.domain.Storage; -import org.zalando.nakadi.domain.Timeline; -import org.zalando.nakadi.exceptions.runtime.ConflictException; +import org.zalando.nakadi.domain.*; import org.zalando.nakadi.exceptions.InternalNakadiException; import org.zalando.nakadi.exceptions.InvalidCursorException; import org.zalando.nakadi.exceptions.NakadiException; import org.zalando.nakadi.exceptions.NoSuchEventTypeException; -import org.zalando.nakadi.exceptions.runtime.NotFoundException; -import org.zalando.nakadi.exceptions.runtime.TimelineException; -import org.zalando.nakadi.exceptions.runtime.TopicCreationException; -import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; -import org.zalando.nakadi.exceptions.runtime.UnableProcessException; -import org.zalando.nakadi.exceptions.runtime.AccessDeniedException; -import org.zalando.nakadi.exceptions.runtime.DbWriteOperationsBlockedException; -import org.zalando.nakadi.exceptions.runtime.DuplicatedTimelineException; -import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; -import org.zalando.nakadi.exceptions.runtime.RepositoryProblemException; -import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; -import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; +import org.zalando.nakadi.exceptions.runtime.*; import org.zalando.nakadi.plugin.api.authz.AuthorizationService; import org.zalando.nakadi.plugin.api.authz.Resource; import org.zalando.nakadi.repository.EventConsumer; @@ -70,6 +52,7 @@ public class TimelineService { private final DefaultStorage defaultStorage; private final AdminService adminService; private final FeatureToggleService featureToggleService; + private final String compactedStorageName; @Autowired public TimelineService(final EventTypeCache eventTypeCache, @@ -80,7 +63,9 @@ public TimelineService(final EventTypeCache eventTypeCache, final TopicRepositoryHolder topicRepositoryHolder, final TransactionTemplate transactionTemplate, @Qualifier("default_storage") final DefaultStorage defaultStorage, - final AdminService adminService, final FeatureToggleService featureToggleService) { + final AdminService adminService, + final FeatureToggleService featureToggleService, + @Value("${timelines.storage.compacted}") final String compactedStorageName) { this.eventTypeCache = eventTypeCache; this.storageDbRepository = storageDbRepository; this.timelineSync = timelineSync; @@ -91,6 +76,7 @@ public TimelineService(final EventTypeCache eventTypeCache, this.defaultStorage = defaultStorage; this.adminService = adminService; this.featureToggleService = featureToggleService; + this.compactedStorageName = compactedStorageName; } public void createTimeline(final String eventTypeName, final String storageId) @@ -118,7 +104,7 @@ public void createTimeline(final String eventTypeName, final String storageId) currentTopicRepo.loadTopicStatistics(Collections.singleton(activeTimeline)); final String newTopic = nextTopicRepo.createTopic(partitionStatistics.size(), - eventType.getOptions().getRetentionTime()); + eventType.getOptions().getRetentionTime(), eventType.getCleanupPolicy()); final Timeline nextTimeline = Timeline.createTimeline(activeTimeline.getEventType(), activeTimeline.getOrder() + 1, storage, newTopic, new Date()); @@ -130,9 +116,7 @@ public void createTimeline(final String eventTypeName, final String storageId) } } - public Timeline createDefaultTimeline(final String eventTypeName, - final int partitionsCount, - final long retentionTime) + public Timeline createDefaultTimeline(final EventTypeBase eventType, final int partitionsCount) throws TopicCreationException, InconsistentStateException, RepositoryProblemException, @@ -143,15 +127,22 @@ public Timeline createDefaultTimeline(final String eventTypeName, throw new DbWriteOperationsBlockedException("Cannot create default timeline: write operations on DB " + "are blocked by feature flag."); } - final TopicRepository repository = topicRepositoryHolder.getTopicRepository(defaultStorage.getStorage()); - final String topic = repository.createTopic(partitionsCount, retentionTime); + + Storage storage = defaultStorage.getStorage(); + if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { + storage = storageDbRepository.getStorage(compactedStorageName).orElseThrow(() -> + new TopicCreationException("No storage defined for compacted topics")); + } + + final TopicRepository repository = topicRepositoryHolder.getTopicRepository(storage); + final String topic = repository.createTopic(partitionsCount, eventType.getOptions().getRetentionTime(), + eventType.getCleanupPolicy()); try { - final Timeline timeline = Timeline.createTimeline(eventTypeName, 1, - defaultStorage.getStorage(), topic, new Date()); + final Timeline timeline = Timeline.createTimeline(eventType.getName(), 1, storage, topic, new Date()); timeline.setSwitchedAt(new Date()); timelineDbRepository.createTimeline(timeline); - eventTypeCache.updated(eventTypeName); + eventTypeCache.updated(eventType.getName()); return timeline; } catch (final InconsistentStateException | RepositoryProblemException | DuplicatedTimelineException e) { rollbackTopic(repository, topic); diff --git a/src/main/resources/application.yml b/src/main/resources/application.yml index 0039ccf508..56749103e6 100644 --- a/src/main/resources/application.yml +++ b/src/main/resources/application.yml @@ -97,12 +97,15 @@ nakadi: deletionDelayMs: 2000 # 2 seconds, to be on the safe side consumerNodesCleanup.runPeriodMs: 21600000 # 6 hours http.pool.connection: - max.total: 20 - max.per.route: 10 - request.timeout: 2000 - connect.timeout: 1000 - socket.timeout: 2000 - timelines.storage.default: "default" + max.total: 20 + max.per.route: 10 + request.timeout: 2000 + connect.timeout: 1000 + socket.timeout: 2000 + timelines: + storage: + default: "default" + compacted: "compacted" kpi: config: batch-collection-timeout: 1000 diff --git a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java index f48cf19ca7..7d95eb2957 100644 --- a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java +++ b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java @@ -1,11 +1,7 @@ package org.zalando.nakadi.controller; import com.google.common.base.Charsets; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; -import com.google.common.collect.Multimap; +import com.google.common.collect.*; import com.google.common.io.Resources; import org.hamcrest.core.StringContains; import org.json.JSONObject; @@ -14,23 +10,11 @@ import org.springframework.test.web.servlet.ResultActions; import org.springframework.test.web.servlet.request.MockHttpServletRequestBuilder; import org.zalando.nakadi.config.SecuritySettings; -import org.zalando.nakadi.domain.EnrichmentStrategyDescriptor; -import org.zalando.nakadi.domain.Audience; -import org.zalando.nakadi.domain.EventType; -import org.zalando.nakadi.domain.EventTypeBase; -import org.zalando.nakadi.domain.EventTypeOptions; -import org.zalando.nakadi.domain.ResourceAuthorization; -import org.zalando.nakadi.domain.ResourceAuthorizationAttribute; -import org.zalando.nakadi.domain.Subscription; -import org.zalando.nakadi.domain.Timeline; +import org.zalando.nakadi.domain.*; import org.zalando.nakadi.exceptions.InternalNakadiException; import org.zalando.nakadi.exceptions.NoSuchEventTypeException; -import org.zalando.nakadi.exceptions.runtime.InvalidEventTypeException; -import org.zalando.nakadi.exceptions.runtime.TopicCreationException; -import org.zalando.nakadi.exceptions.runtime.UnableProcessException; import org.zalando.nakadi.exceptions.UnprocessableEntityException; -import org.zalando.nakadi.exceptions.runtime.DuplicatedEventTypeNameException; -import org.zalando.nakadi.exceptions.runtime.TopicConfigException; +import org.zalando.nakadi.exceptions.runtime.*; import org.zalando.nakadi.partitioning.PartitionStrategy; import org.zalando.nakadi.repository.TopicRepository; import org.zalando.nakadi.utils.EventTypeTestBuilder; @@ -41,11 +25,7 @@ import javax.ws.rs.core.Response; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Date; -import java.util.List; -import java.util.Optional; +import java.util.*; import java.util.concurrent.TimeoutException; import static javax.ws.rs.core.Response.Status.NOT_FOUND; @@ -57,24 +37,12 @@ import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; import static org.springframework.http.MediaType.APPLICATION_JSON; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; -import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; -import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.header; -import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.*; import static org.zalando.nakadi.domain.EventCategory.BUSINESS; -import static org.zalando.nakadi.utils.TestUtils.buildDefaultEventType; -import static org.zalando.nakadi.utils.TestUtils.buildTimelineWithTopic; -import static org.zalando.nakadi.utils.TestUtils.createInvalidEventTypeExceptionProblem; -import static org.zalando.nakadi.utils.TestUtils.invalidProblem; -import static org.zalando.nakadi.utils.TestUtils.randomValidEventTypeName; +import static org.zalando.nakadi.utils.TestUtils.*; public class EventTypeControllerTest extends EventTypeControllerTestCase { @@ -122,7 +90,7 @@ public void whenPostEventTypeWithCorrectNameThen201() throws Exception { public void whenPostEventTypeThenWarning() throws Exception { final EventType eventType = buildDefaultEventType(); postEventType(eventType).andExpect(status().isCreated()).andExpect( - header().string("Warning","299 nakadi \"I am warning you\"")); + header().string("Warning", "299 nakadi \"I am warning you\"")); } @Test @@ -248,7 +216,7 @@ public void whenPUTthenWarning() throws Exception { doReturn(eventType).when(eventTypeRepository).findByName(any()); putEventType(updatedEventType, eventType.getName()).andExpect( - header().string("Warning","299 nakadi \"I am warning you\"")); + header().string("Warning", "299 nakadi \"I am warning you\"")); } @Test @@ -420,7 +388,7 @@ public void whenPostWithValidAuthorizationThenCreated() throws Exception { ImmutableList.of(new ResourceAuthorizationAttribute("type3", "value3")))); doReturn(eventType).when(eventTypeRepository).saveEventType(any(EventType.class)); - when(topicRepository.createTopic(anyInt(), any())).thenReturn(randomUUID.toString()); + when(topicRepository.createTopic(anyInt(), any(), any())).thenReturn(randomUUID.toString()); postEventType(eventType).andExpect(status().isCreated()); } @@ -627,13 +595,13 @@ public void whenPersistencyErrorThen500() throws Exception { public void whenCreateSuccessfullyThen201() throws Exception { final EventType et = buildDefaultEventType(); final Timeline timeline = buildTimelineWithTopic("topic1"); - when(timelineService.createDefaultTimeline(anyString(), anyInt(), anyLong())).thenReturn(timeline); + when(timelineService.createDefaultTimeline(any(), anyInt())).thenReturn(timeline); doReturn(et).when(eventTypeRepository).saveEventType(any(EventType.class)); postEventType(et).andExpect(status().isCreated()).andExpect(content().string("")); verify(eventTypeRepository, times(1)).saveEventType(any(EventType.class)); - verify(timelineService, times(1)).createDefaultTimeline(anyString(), anyInt(), anyLong()); + verify(timelineService, times(1)).createDefaultTimeline(any(), anyInt()); } @Test @@ -641,7 +609,7 @@ public void whenTimelineCreationFailsRemoveEventTypeFromRepositoryAnd500() throw final EventType et = buildDefaultEventType(); doThrow(TopicCreationException.class).when(timelineService) - .createDefaultTimeline(anyString(), anyInt(), anyLong()); + .createDefaultTimeline(any(), anyInt()); final Problem expectedProblem = Problem.valueOf(SERVICE_UNAVAILABLE); postEventType(et).andExpect(status().isServiceUnavailable()) @@ -649,7 +617,7 @@ public void whenTimelineCreationFailsRemoveEventTypeFromRepositoryAnd500() throw matchesProblem(expectedProblem))); verify(eventTypeRepository, times(1)).saveEventType(any(EventType.class)); - verify(timelineService, times(1)).createDefaultTimeline(anyString(), anyInt(), anyLong()); + verify(timelineService, times(1)).createDefaultTimeline(any(), anyInt()); verify(eventTypeRepository, times(1)).removeEventType(et.getName()); } diff --git a/src/test/java/org/zalando/nakadi/service/EventTypeServiceTest.java b/src/test/java/org/zalando/nakadi/service/EventTypeServiceTest.java index cc962f5768..207c17e5e6 100644 --- a/src/test/java/org/zalando/nakadi/service/EventTypeServiceTest.java +++ b/src/test/java/org/zalando/nakadi/service/EventTypeServiceTest.java @@ -33,15 +33,7 @@ import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyZeroInteractions; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; import static org.zalando.nakadi.utils.TestUtils.buildDefaultEventType; import static org.zalando.nakadi.utils.TestUtils.checkKPIEventSubmitted; @@ -135,11 +127,13 @@ public void testFeatureToggleAllowsDeleteEventTypeWithSubscriptions() throws Exc @Test public void shouldRemoveEventTypeWhenTimelineCreationFails() throws Exception { final EventType eventType = buildDefaultEventType(); - when(timelineService.createDefaultTimeline(anyString(), anyInt(), anyLong())) + when(timelineService.createDefaultTimeline(any(), anyInt())) .thenThrow(new TopicCreationException("Failed to create topic")); try { eventTypeService.create(eventType); + fail("should throw TopicCreationException"); } catch (final TopicCreationException e) { + // expected } verify(eventTypeRepository, times(1)).removeEventType(eventType.getName()); diff --git a/src/test/java/org/zalando/nakadi/service/timeline/TimelineServiceTest.java b/src/test/java/org/zalando/nakadi/service/timeline/TimelineServiceTest.java index ef85d9dfc5..d145704682 100644 --- a/src/test/java/org/zalando/nakadi/service/timeline/TimelineServiceTest.java +++ b/src/test/java/org/zalando/nakadi/service/timeline/TimelineServiceTest.java @@ -13,9 +13,9 @@ import org.zalando.nakadi.domain.Timeline; import org.zalando.nakadi.exceptions.InternalNakadiException; import org.zalando.nakadi.exceptions.NoSuchEventTypeException; +import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; import org.zalando.nakadi.exceptions.runtime.NotFoundException; import org.zalando.nakadi.exceptions.runtime.TimelineException; -import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; import org.zalando.nakadi.repository.TopicRepository; import org.zalando.nakadi.repository.TopicRepositoryHolder; import org.zalando.nakadi.repository.db.EventTypeCache; @@ -47,7 +47,7 @@ public class TimelineServiceTest { private final TimelineService timelineService = new TimelineService(eventTypeCache, storageDbRepository, mock(TimelineSync.class), mock(NakadiSettings.class), timelineDbRepository, topicRepositoryHolder, new TransactionTemplate(mock(PlatformTransactionManager.class)), - new DefaultStorage(new Storage()), adminService, featureToggleService); + new DefaultStorage(new Storage()), adminService, featureToggleService, "compacted-storage"); @Test(expected = NotFoundException.class) public void testGetTimelinesNotFound() throws Exception { @@ -115,7 +115,7 @@ public void shouldDeleteTopicWhenTimelineCreationFails() throws Exception { Mockito.when(timelineDbRepository.createTimeline(any())) .thenThrow(new InconsistentStateException("shouldDeleteTopicWhenTimelineCreationFails")); try { - timelineService.createDefaultTimeline("event_type_1", 1, 1); + timelineService.createDefaultTimeline(buildDefaultEventType(), 1); } catch (final InconsistentStateException e) { } From ebb0d367a7c42b2a1cca582546d9a7b4e22b42f6 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Wed, 4 Jul 2018 17:10:46 +0200 Subject: [PATCH 02/26] ARUHA-1757: fixed imports; --- .../repository/kafka/KafkaRepositoryAT.java | 6 ++- .../nakadi/repository/TopicRepository.java | 14 +++++- .../kafka/KafkaTopicRepository.java | 45 +++++++++++++++--- .../nakadi/service/EventPublisher.java | 24 ++++++++-- .../nakadi/service/EventTypeService.java | 34 ++++++++++++-- .../service/timeline/TimelineService.java | 24 +++++++++- .../controller/EventTypeControllerTest.java | 46 ++++++++++++++++--- .../nakadi/service/EventTypeServiceTest.java | 8 +++- 8 files changed, 176 insertions(+), 25 deletions(-) diff --git a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java index 0aacf9afda..99c21df78e 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java @@ -28,7 +28,11 @@ import static org.echocat.jomon.runtime.concurrent.Retryer.executeWithRetry; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.*; +import static org.hamcrest.Matchers.arrayWithSize; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.not; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/src/main/java/org/zalando/nakadi/repository/TopicRepository.java b/src/main/java/org/zalando/nakadi/repository/TopicRepository.java index aeee847b32..5cd86cced9 100644 --- a/src/main/java/org/zalando/nakadi/repository/TopicRepository.java +++ b/src/main/java/org/zalando/nakadi/repository/TopicRepository.java @@ -1,9 +1,19 @@ package org.zalando.nakadi.repository; -import org.zalando.nakadi.domain.*; +import org.zalando.nakadi.domain.BatchItem; +import org.zalando.nakadi.domain.CleanupPolicy; +import org.zalando.nakadi.domain.NakadiCursor; +import org.zalando.nakadi.domain.PartitionEndStatistics; +import org.zalando.nakadi.domain.PartitionStatistics; +import org.zalando.nakadi.domain.Timeline; import org.zalando.nakadi.exceptions.InvalidCursorException; import org.zalando.nakadi.exceptions.NakadiException; -import org.zalando.nakadi.exceptions.runtime.*; +import org.zalando.nakadi.exceptions.runtime.EventPublishingException; +import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; +import org.zalando.nakadi.exceptions.runtime.TopicConfigException; +import org.zalando.nakadi.exceptions.runtime.TopicCreationException; +import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; +import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; import java.util.Collection; import java.util.List; diff --git a/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java index f37a911bb0..e59253d5ad 100644 --- a/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java +++ b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java @@ -10,15 +10,32 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.*; +import org.apache.kafka.common.errors.InterruptException; +import org.apache.kafka.common.errors.NetworkException; +import org.apache.kafka.common.errors.NotLeaderForPartitionException; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.echocat.jomon.runtime.concurrent.RetryForSpecifiedTimeStrategy; import org.echocat.jomon.runtime.concurrent.Retryer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.zalando.nakadi.config.NakadiSettings; -import org.zalando.nakadi.domain.*; +import org.zalando.nakadi.domain.BatchItem; +import org.zalando.nakadi.domain.CleanupPolicy; +import org.zalando.nakadi.domain.EventPublishingStatus; +import org.zalando.nakadi.domain.EventPublishingStep; +import org.zalando.nakadi.domain.NakadiCursor; +import org.zalando.nakadi.domain.PartitionEndStatistics; +import org.zalando.nakadi.domain.PartitionStatistics; +import org.zalando.nakadi.domain.Timeline; import org.zalando.nakadi.exceptions.InvalidCursorException; -import org.zalando.nakadi.exceptions.runtime.*; +import org.zalando.nakadi.exceptions.runtime.EventPublishingException; +import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; +import org.zalando.nakadi.exceptions.runtime.TopicConfigException; +import org.zalando.nakadi.exceptions.runtime.TopicCreationException; +import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; +import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; import org.zalando.nakadi.repository.EventConsumer; import org.zalando.nakadi.repository.TopicRepository; import org.zalando.nakadi.repository.zookeeper.ZooKeeperHolder; @@ -26,8 +43,21 @@ import org.zalando.nakadi.util.UUIDGenerator; import javax.annotation.Nullable; -import java.util.*; -import java.util.concurrent.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -36,7 +66,10 @@ import static com.google.common.collect.Lists.newArrayList; import static java.util.Collections.unmodifiableList; import static java.util.stream.Collectors.toList; -import static org.zalando.nakadi.domain.CursorError.*; +import static org.zalando.nakadi.domain.CursorError.NULL_OFFSET; +import static org.zalando.nakadi.domain.CursorError.NULL_PARTITION; +import static org.zalando.nakadi.domain.CursorError.PARTITION_NOT_FOUND; +import static org.zalando.nakadi.domain.CursorError.UNAVAILABLE; public class KafkaTopicRepository implements TopicRepository { diff --git a/src/main/java/org/zalando/nakadi/service/EventPublisher.java b/src/main/java/org/zalando/nakadi/service/EventPublisher.java index 33fc9d224a..25572fc465 100644 --- a/src/main/java/org/zalando/nakadi/service/EventPublisher.java +++ b/src/main/java/org/zalando/nakadi/service/EventPublisher.java @@ -7,10 +7,28 @@ import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; import org.zalando.nakadi.config.NakadiSettings; -import org.zalando.nakadi.domain.*; +import org.zalando.nakadi.domain.BatchFactory; +import org.zalando.nakadi.domain.BatchItem; +import org.zalando.nakadi.domain.BatchItemResponse; +import org.zalando.nakadi.domain.CleanupPolicy; +import org.zalando.nakadi.domain.EventCategory; +import org.zalando.nakadi.domain.EventPublishResult; +import org.zalando.nakadi.domain.EventPublishingStatus; +import org.zalando.nakadi.domain.EventPublishingStep; +import org.zalando.nakadi.domain.EventType; +import org.zalando.nakadi.domain.Timeline; import org.zalando.nakadi.enrichment.Enrichment; -import org.zalando.nakadi.exceptions.*; -import org.zalando.nakadi.exceptions.runtime.*; +import org.zalando.nakadi.exceptions.EnrichmentException; +import org.zalando.nakadi.exceptions.InternalNakadiException; +import org.zalando.nakadi.exceptions.InvalidPartitionKeyFieldsException; +import org.zalando.nakadi.exceptions.NoSuchEventTypeException; +import org.zalando.nakadi.exceptions.PartitioningException; +import org.zalando.nakadi.exceptions.runtime.AccessDeniedException; +import org.zalando.nakadi.exceptions.runtime.EventPublishingException; +import org.zalando.nakadi.exceptions.runtime.EventTypeTimeoutException; +import org.zalando.nakadi.exceptions.runtime.EventValidationException; +import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; +import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; import org.zalando.nakadi.partitioning.PartitionResolver; import org.zalando.nakadi.repository.db.EventTypeCache; import org.zalando.nakadi.service.timeline.TimelineService; diff --git a/src/main/java/org/zalando/nakadi/service/EventTypeService.java b/src/main/java/org/zalando/nakadi/service/EventTypeService.java index 8e4002f748..9037e126b8 100644 --- a/src/main/java/org/zalando/nakadi/service/EventTypeService.java +++ b/src/main/java/org/zalando/nakadi/service/EventTypeService.java @@ -15,10 +15,38 @@ import org.springframework.transaction.TransactionException; import org.springframework.transaction.support.TransactionTemplate; import org.zalando.nakadi.config.NakadiSettings; -import org.zalando.nakadi.domain.*; +import org.zalando.nakadi.domain.CompatibilityMode; +import org.zalando.nakadi.domain.EventCategory; +import org.zalando.nakadi.domain.EventType; +import org.zalando.nakadi.domain.EventTypeBase; +import org.zalando.nakadi.domain.EventTypeOptions; +import org.zalando.nakadi.domain.EventTypeStatistics; +import org.zalando.nakadi.domain.Subscription; +import org.zalando.nakadi.domain.Timeline; import org.zalando.nakadi.enrichment.Enrichment; -import org.zalando.nakadi.exceptions.*; -import org.zalando.nakadi.exceptions.runtime.*; +import org.zalando.nakadi.exceptions.InternalNakadiException; +import org.zalando.nakadi.exceptions.NakadiException; +import org.zalando.nakadi.exceptions.NakadiRuntimeException; +import org.zalando.nakadi.exceptions.NoSuchEventTypeException; +import org.zalando.nakadi.exceptions.NoSuchPartitionStrategyException; +import org.zalando.nakadi.exceptions.NoSuchSubscriptionException; +import org.zalando.nakadi.exceptions.runtime.AccessDeniedException; +import org.zalando.nakadi.exceptions.runtime.ConflictException; +import org.zalando.nakadi.exceptions.runtime.DbWriteOperationsBlockedException; +import org.zalando.nakadi.exceptions.runtime.DuplicatedEventTypeNameException; +import org.zalando.nakadi.exceptions.runtime.EventTypeDeletionException; +import org.zalando.nakadi.exceptions.runtime.EventTypeOptionsValidationException; +import org.zalando.nakadi.exceptions.runtime.EventTypeUnavailableException; +import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; +import org.zalando.nakadi.exceptions.runtime.InvalidEventTypeException; +import org.zalando.nakadi.exceptions.runtime.NoEventTypeException; +import org.zalando.nakadi.exceptions.runtime.NotFoundException; +import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; +import org.zalando.nakadi.exceptions.runtime.TimelineException; +import org.zalando.nakadi.exceptions.runtime.TopicConfigException; +import org.zalando.nakadi.exceptions.runtime.TopicCreationException; +import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; +import org.zalando.nakadi.exceptions.runtime.UnableProcessException; import org.zalando.nakadi.partitioning.PartitionResolver; import org.zalando.nakadi.plugin.api.authz.AuthorizationService; import org.zalando.nakadi.repository.EventTypeRepository; diff --git a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java index 81f43eb411..1b313fec88 100644 --- a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java +++ b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java @@ -11,12 +11,32 @@ import org.springframework.transaction.TransactionException; import org.springframework.transaction.support.TransactionTemplate; import org.zalando.nakadi.config.NakadiSettings; -import org.zalando.nakadi.domain.*; +import org.zalando.nakadi.domain.CleanupPolicy; +import org.zalando.nakadi.domain.DefaultStorage; +import org.zalando.nakadi.domain.EventType; +import org.zalando.nakadi.domain.EventTypeBase; +import org.zalando.nakadi.domain.EventTypeResource; +import org.zalando.nakadi.domain.NakadiCursor; +import org.zalando.nakadi.domain.PartitionStatistics; +import org.zalando.nakadi.domain.Storage; +import org.zalando.nakadi.domain.Timeline; import org.zalando.nakadi.exceptions.InternalNakadiException; import org.zalando.nakadi.exceptions.InvalidCursorException; import org.zalando.nakadi.exceptions.NakadiException; import org.zalando.nakadi.exceptions.NoSuchEventTypeException; -import org.zalando.nakadi.exceptions.runtime.*; +import org.zalando.nakadi.exceptions.runtime.AccessDeniedException; +import org.zalando.nakadi.exceptions.runtime.ConflictException; +import org.zalando.nakadi.exceptions.runtime.DbWriteOperationsBlockedException; +import org.zalando.nakadi.exceptions.runtime.DuplicatedTimelineException; +import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; +import org.zalando.nakadi.exceptions.runtime.NotFoundException; +import org.zalando.nakadi.exceptions.runtime.RepositoryProblemException; +import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; +import org.zalando.nakadi.exceptions.runtime.TimelineException; +import org.zalando.nakadi.exceptions.runtime.TopicCreationException; +import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; +import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; +import org.zalando.nakadi.exceptions.runtime.UnableProcessException; import org.zalando.nakadi.plugin.api.authz.AuthorizationService; import org.zalando.nakadi.plugin.api.authz.Resource; import org.zalando.nakadi.repository.EventConsumer; diff --git a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java index 7d95eb2957..329787e1a0 100644 --- a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java +++ b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java @@ -1,7 +1,11 @@ package org.zalando.nakadi.controller; import com.google.common.base.Charsets; -import com.google.common.collect.*; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Multimap; import com.google.common.io.Resources; import org.hamcrest.core.StringContains; import org.json.JSONObject; @@ -10,11 +14,23 @@ import org.springframework.test.web.servlet.ResultActions; import org.springframework.test.web.servlet.request.MockHttpServletRequestBuilder; import org.zalando.nakadi.config.SecuritySettings; -import org.zalando.nakadi.domain.*; +import org.zalando.nakadi.domain.Audience; +import org.zalando.nakadi.domain.EnrichmentStrategyDescriptor; +import org.zalando.nakadi.domain.EventType; +import org.zalando.nakadi.domain.EventTypeBase; +import org.zalando.nakadi.domain.EventTypeOptions; +import org.zalando.nakadi.domain.ResourceAuthorization; +import org.zalando.nakadi.domain.ResourceAuthorizationAttribute; +import org.zalando.nakadi.domain.Subscription; +import org.zalando.nakadi.domain.Timeline; import org.zalando.nakadi.exceptions.InternalNakadiException; import org.zalando.nakadi.exceptions.NoSuchEventTypeException; import org.zalando.nakadi.exceptions.UnprocessableEntityException; -import org.zalando.nakadi.exceptions.runtime.*; +import org.zalando.nakadi.exceptions.runtime.DuplicatedEventTypeNameException; +import org.zalando.nakadi.exceptions.runtime.InvalidEventTypeException; +import org.zalando.nakadi.exceptions.runtime.TopicConfigException; +import org.zalando.nakadi.exceptions.runtime.TopicCreationException; +import org.zalando.nakadi.exceptions.runtime.UnableProcessException; import org.zalando.nakadi.partitioning.PartitionStrategy; import org.zalando.nakadi.repository.TopicRepository; import org.zalando.nakadi.utils.EventTypeTestBuilder; @@ -25,7 +41,11 @@ import javax.ws.rs.core.Response; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Date; +import java.util.List; +import java.util.Optional; import java.util.concurrent.TimeoutException; import static javax.ws.rs.core.Response.Status.NOT_FOUND; @@ -37,12 +57,24 @@ import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import static org.springframework.http.MediaType.APPLICATION_JSON; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; -import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.*; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.header; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; import static org.zalando.nakadi.domain.EventCategory.BUSINESS; -import static org.zalando.nakadi.utils.TestUtils.*; +import static org.zalando.nakadi.utils.TestUtils.buildDefaultEventType; +import static org.zalando.nakadi.utils.TestUtils.buildTimelineWithTopic; +import static org.zalando.nakadi.utils.TestUtils.createInvalidEventTypeExceptionProblem; +import static org.zalando.nakadi.utils.TestUtils.invalidProblem; +import static org.zalando.nakadi.utils.TestUtils.randomValidEventTypeName; public class EventTypeControllerTest extends EventTypeControllerTestCase { diff --git a/src/test/java/org/zalando/nakadi/service/EventTypeServiceTest.java b/src/test/java/org/zalando/nakadi/service/EventTypeServiceTest.java index 207c17e5e6..bb2d25c993 100644 --- a/src/test/java/org/zalando/nakadi/service/EventTypeServiceTest.java +++ b/src/test/java/org/zalando/nakadi/service/EventTypeServiceTest.java @@ -33,7 +33,13 @@ import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; import static org.zalando.nakadi.utils.TestUtils.buildDefaultEventType; import static org.zalando.nakadi.utils.TestUtils.checkKPIEventSubmitted; From 51de225b4da13a288d199085ff9bdfb16ee98d14 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Thu, 5 Jul 2018 11:26:39 +0200 Subject: [PATCH 03/26] ARUHA-1757: fixed codestyle and tests; --- .../java/org/zalando/nakadi/domain/EventTypeBase.java | 10 +++++++--- .../nakadi/service/timeline/TimelineService.java | 2 +- .../zalando/nakadi/security/AuthenticationTest.java | 7 ++++--- .../org/zalando/nakadi/utils/EventTypeTestBuilder.java | 10 +++++++++- 4 files changed, 21 insertions(+), 8 deletions(-) diff --git a/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java b/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java index 5d99e09856..d74319ebed 100644 --- a/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java +++ b/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java @@ -90,7 +90,8 @@ public EventTypeBase(final String name, final String owningApplication, final List partitionKeyFields, final EventTypeSchemaBase schema, final EventTypeStatistics defaultStatistic, final EventTypeOptions options, - final CompatibilityMode compatibilityMode) { + final CompatibilityMode compatibilityMode, + final CleanupPolicy cleanupPolicy) { this.name = name; this.owningApplication = owningApplication; this.category = category; @@ -102,6 +103,7 @@ public EventTypeBase(final String name, final String owningApplication, this.defaultStatistic = defaultStatistic; this.options = options; this.compatibilityMode = compatibilityMode; + this.cleanupPolicy = cleanupPolicy; } public EventTypeBase(final EventTypeBase eventType) { @@ -119,6 +121,8 @@ public EventTypeBase(final EventTypeBase eventType) { this.setAuthorization(eventType.getAuthorization()); this.setAudience(eventType.getAudience()); this.setOrderingKeyFields(eventType.getOrderingKeyFields()); + this.setCleanupPolicy(eventType.getCleanupPolicy()); + this.setPartitionCompactionKeys(eventType.getPartitionCompactionKeys()); } public String getName() { @@ -185,7 +189,7 @@ public CleanupPolicy getCleanupPolicy() { return cleanupPolicy; } - public void setCleanupPolicy(CleanupPolicy cleanupPolicy) { + public void setCleanupPolicy(final CleanupPolicy cleanupPolicy) { this.cleanupPolicy = cleanupPolicy; } @@ -194,7 +198,7 @@ public List getPartitionCompactionKeys() { EMPTY_PARTITION_COMPACTION_KEYS); } - public void setPartitionCompactionKeys(@Nullable List partitionCompactionKeys) { + public void setPartitionCompactionKeys(@Nullable final List partitionCompactionKeys) { this.partitionCompactionKeys = partitionCompactionKeys; } diff --git a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java index 1b313fec88..cd32a523d2 100644 --- a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java +++ b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java @@ -85,7 +85,7 @@ public TimelineService(final EventTypeCache eventTypeCache, @Qualifier("default_storage") final DefaultStorage defaultStorage, final AdminService adminService, final FeatureToggleService featureToggleService, - @Value("${timelines.storage.compacted}") final String compactedStorageName) { + @Value("${nakadi.timelines.storage.compacted}") final String compactedStorageName) { this.eventTypeCache = eventTypeCache; this.storageDbRepository = storageDbRepository; this.timelineSync = timelineSync; diff --git a/src/test/java/org/zalando/nakadi/security/AuthenticationTest.java b/src/test/java/org/zalando/nakadi/security/AuthenticationTest.java index 6f845ba0c8..8e43034745 100644 --- a/src/test/java/org/zalando/nakadi/security/AuthenticationTest.java +++ b/src/test/java/org/zalando/nakadi/security/AuthenticationTest.java @@ -45,8 +45,8 @@ import org.zalando.nakadi.service.EventPublisher; import org.zalando.nakadi.service.EventStreamFactory; import org.zalando.nakadi.service.EventTypeService; -import org.zalando.nakadi.service.timeline.TimelineSync; import org.zalando.nakadi.service.FeatureToggleService; +import org.zalando.nakadi.service.timeline.TimelineSync; import org.zalando.nakadi.util.UUIDGenerator; import org.zalando.stups.oauth2.spring.security.expression.ExtendedOAuth2WebSecurityExpressionHandler; @@ -76,7 +76,7 @@ import static org.zalando.nakadi.utils.TestUtils.randomUUID; @RunWith(SpringJUnit4ClassRunner.class) -@SpringBootTest(classes=Application.class, webEnvironment= WebEnvironment.RANDOM_PORT) +@SpringBootTest(classes = Application.class, webEnvironment = WebEnvironment.RANDOM_PORT) @DirtiesContext(classMode = AFTER_CLASS) @ActiveProfiles("test") public abstract class AuthenticationTest { @@ -165,6 +165,7 @@ public EventTypeRepository mockDbRepository() { public SubscriptionDbRepository mockSubscriptionDbRepo() { return mock(SubscriptionDbRepository.class); } + @Bean public EventTypeCache eventTypeCache() { return mock(EventTypeCache.class); @@ -265,7 +266,7 @@ public MessageDigest sha256MessageDigest() { } protected static final ResultMatcher STATUS_NOT_401_OR_403 = status().is(not( - isOneOf(UNAUTHORIZED.value(), FORBIDDEN.value()))); + isOneOf(UNAUTHORIZED.value(), FORBIDDEN.value()))); protected static final String TOKEN_WITH_UID_SCOPE = randomUUID(); protected static final String TOKEN_WITH_NAKADI_ADMIN_SCOPE = randomUUID(); diff --git a/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java b/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java index 710ea982a7..f3f177bd30 100644 --- a/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java +++ b/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java @@ -4,6 +4,7 @@ import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.json.JSONObject; +import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.CompatibilityMode; import org.zalando.nakadi.domain.EnrichmentStrategyDescriptor; import org.zalando.nakadi.domain.Audience; @@ -37,6 +38,7 @@ public class EventTypeTestBuilder { private EventTypeStatistics defaultStatistic; private EventTypeOptions options; private CompatibilityMode compatibilityMode; + private CleanupPolicy cleanupPolicy; private DateTime createdAt; private DateTime updatedAt; private Audience audience; @@ -56,6 +58,7 @@ public EventTypeTestBuilder() { this.options = new EventTypeOptions(); this.options.setRetentionTime(172800000L); this.compatibilityMode = CompatibilityMode.COMPATIBLE; + this.cleanupPolicy = CleanupPolicy.DELETE; this.createdAt = new DateTime(DateTimeZone.UTC); this.updatedAt = this.createdAt; this.authorization = null; @@ -128,6 +131,11 @@ public EventTypeTestBuilder compatibilityMode(final CompatibilityMode compatibil return this; } + public EventTypeTestBuilder cleanupPolicy(final CleanupPolicy cleanupPolicy) { + this.cleanupPolicy = cleanupPolicy; + return this; + } + public EventTypeTestBuilder createdAt(final DateTime createdAt) { this.createdAt = createdAt; return this; @@ -151,7 +159,7 @@ public EventTypeTestBuilder audience(final Audience audience) { public EventType build() { final EventTypeBase eventTypeBase = new EventTypeBase(name, owningApplication, category, validationStrategies, enrichmentStrategies, partitionStrategy, partitionKeyFields, schema, - defaultStatistic, options, compatibilityMode); + defaultStatistic, options, compatibilityMode, cleanupPolicy); eventTypeBase.setAuthorization(authorization); eventTypeBase.setAudience(audience); return new EventType(eventTypeBase, this.schema.getVersion().toString(), this.createdAt, this.updatedAt); From fe36b2af2577c42dbad911dbbf0be0337a510dce Mon Sep 17 00:00:00 2001 From: vstepanov Date: Fri, 6 Jul 2018 14:54:10 +0200 Subject: [PATCH 04/26] ARUHA-1757: created tests for log compaction; --- .../repository/kafka/KafkaRepositoryAT.java | 34 +++++++++++++++---- .../repository/kafka/KafkaTestHelper.java | 16 ++++++--- .../nakadi/webservice/EventTypeAT.java | 24 +++++++++++++ .../nakadi/webservice/UserJourneyAT.java | 2 ++ src/main/resources/application.yml | 2 +- 5 files changed, 65 insertions(+), 13 deletions(-) diff --git a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java index 99c21df78e..cd5aee2b5c 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java @@ -36,6 +36,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.zalando.nakadi.utils.TestUtils.waitFor; public class KafkaRepositoryAT extends BaseAT { @@ -48,9 +49,9 @@ public class KafkaRepositoryAT extends BaseAT { private static final int ZK_CONNECTION_TIMEOUT = 10000; private static final int NAKADI_SEND_TIMEOUT = 10000; private static final int NAKADI_POLL_TIMEOUT = 10000; - private static final Long RETENTION_TIME = 100L; + private static final Long DEFAULT_RETENTION_TIME = 100L; private static final Long DEFAULT_TOPIC_RETENTION = 100000000L; - private static final CleanupPolicy CLEANUP_POLICY = CleanupPolicy.DELETE; + private static final CleanupPolicy DEFAULT_CLEANUP_POLICY = CleanupPolicy.DELETE; private static final int KAFKA_REQUEST_TIMEOUT = 30000; private static final int KAFKA_BATCH_SIZE = 1048576; private static final long KAFKA_LINGER_MS = 0; @@ -97,8 +98,8 @@ public void setup() { @SuppressWarnings("unchecked") public void whenCreateTopicThenTopicIsCreated() throws Exception { // ACT // - final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, RETENTION_TIME, - CLEANUP_POLICY); + final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, DEFAULT_RETENTION_TIME, + DEFAULT_CLEANUP_POLICY); // ASSERT // executeWithRetry(() -> { @@ -165,17 +166,36 @@ public void whenBulkSendSuccessfullyThenUpdateBatchItemStatus() throws Exception @SuppressWarnings("unchecked") public void whenCreateTopicWithRetentionTime() throws Exception { // ACT // - final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, RETENTION_TIME, - CLEANUP_POLICY); + final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, DEFAULT_RETENTION_TIME, + DEFAULT_CLEANUP_POLICY); // ASSERT // executeWithRetry(() -> Assert.assertEquals( - KafkaTestHelper.getTopicRetentionTime(topicName, ZOOKEEPER_URL), RETENTION_TIME), + KafkaTestHelper.getTopicRetentionTime(topicName, ZOOKEEPER_URL), DEFAULT_RETENTION_TIME), new RetryForSpecifiedTimeStrategy(5000) .withExceptionsThatForceRetry(AssertionError.class) .withWaitBetweenEachTry(500)); } + @Test(timeout = 10000) + @SuppressWarnings("unchecked") + public void checkCompactionCleanupPolicySetCorrectly() { + setCleanupPolicyAndCheck(CleanupPolicy.DELETE, "delete"); + setCleanupPolicyAndCheck(CleanupPolicy.COMPACT, "compact"); + } + + private void setCleanupPolicyAndCheck(final CleanupPolicy cleanupPolicy, final String expectedCleanupPolicy) { + // ACT // + final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, DEFAULT_RETENTION_TIME, + cleanupPolicy); + + // ASSERT // + waitFor(() -> { + final String actualCleanupPolicy = KafkaTestHelper.getTopicCleanupPolicy(topicName, ZOOKEEPER_URL); + assertThat(actualCleanupPolicy, equalTo(expectedCleanupPolicy)); + }, 5000); + } + private Map> getAllTopics() { final KafkaConsumer kafkaConsumer = kafkaHelper.createConsumer(); return kafkaConsumer.listTopics(); diff --git a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaTestHelper.java b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaTestHelper.java index 76654bbfb0..6bf1462b62 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaTestHelper.java +++ b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaTestHelper.java @@ -68,8 +68,7 @@ public List getOffsetsToReadFromLatest(final String topic) { .map(cursor -> { if ("0".equals(cursor.getOffset())) { return new Cursor(cursor.getPartition(), "001-0001--1"); - } - else { + } else { final long lastEventOffset = toKafkaOffset(cursor.getOffset()) - 1; final String offset = StringUtils.leftPad(toNakadiOffset(lastEventOffset), CURSOR_OFFSET_LENGTH, '0'); @@ -103,8 +102,7 @@ public void createTopic(final String topic, final String zkUrl) { try { zkUtils = ZkUtils.apply(zkUrl, 30000, 10000, false); AdminUtils.createTopic(zkUtils, topic, 1, 1, new Properties(), RackAwareMode.Safe$.MODULE$); - } - finally { + } finally { if (zkUtils != null) { zkUtils.close(); } @@ -112,8 +110,16 @@ public void createTopic(final String topic, final String zkUrl) { } public static Long getTopicRetentionTime(final String topic, final String zkPath) { + return Long.valueOf(getTopicProperty(topic, zkPath, "retention.ms")); + } + + public static String getTopicCleanupPolicy(final String topic, final String zkPath) { + return getTopicProperty(topic, zkPath, "cleanup.policy"); + } + + private static String getTopicProperty(final String topic, final String zkPath, final String propertyName) { final ZkUtils zkUtils = ZkUtils.apply(zkPath, 30000, 10000, false); final Properties topicConfig = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic(), topic); - return Long.valueOf(topicConfig.getProperty("retention.ms")); + return topicConfig.getProperty(propertyName); } } diff --git a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java index d851b2228e..66de38cf93 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java @@ -7,6 +7,7 @@ import org.json.JSONObject; import org.junit.Assert; import org.junit.Test; +import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.EventType; import org.zalando.nakadi.domain.ResourceAuthorization; import org.zalando.nakadi.domain.ResourceAuthorizationAttribute; @@ -221,6 +222,29 @@ public void whenUpdateRetentionTimeThenUpdateInKafkaAndDB() throws Exception { assertRetentionTime(newRetentionTime, eventType.getName()); } + @Test + public void whenPostCompactedEventTypeThenOk() throws JsonProcessingException { + final EventType eventType = buildDefaultEventType(); + eventType.setCleanupPolicy(CleanupPolicy.COMPACT); + eventType.setPartitionCompactionKeys(ImmutableList.of("key1")); + eventType.getSchema().setSchema("{\"type\":\"object\",\"properties\":{\"key1\":{\"type\":\"string\"}}}"); + + final String body = MAPPER.writer().writeValueAsString(eventType); + given().body(body) + .contentType(JSON) + .post(ENDPOINT) + .then() + .statusCode(HttpStatus.SC_CREATED); + + given().body(body) + .contentType(JSON) + .get(ENDPOINT + "/" + eventType.getName()) + .then() + .statusCode(HttpStatus.SC_OK) + .body("cleanup_policy", equalTo("compact")) + .body("partition_compaction_keys", equalTo(ImmutableList.of("key1"))); + } + @Test public void whenUpdateRetentionTimeWithNullValueNoChange() throws Exception { final EventType eventType = NakadiTestUtils.createEventType(); diff --git a/src/acceptance-test/java/org/zalando/nakadi/webservice/UserJourneyAT.java b/src/acceptance-test/java/org/zalando/nakadi/webservice/UserJourneyAT.java index ca7cbf3caf..91968aef60 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/webservice/UserJourneyAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/webservice/UserJourneyAT.java @@ -92,6 +92,8 @@ public void userJourneyM1() throws InterruptedException, IOException { .body("category", equalTo("undefined")) .body("audience", equalTo("external-public")) .body("ordering_key_fields", equalTo(Lists.newArrayList("foo", "bar.baz"))) + .body("cleanup_policy", equalTo("delete")) + .body("partition_compaction_keys", equalTo(ImmutableList.of())) .body("schema.type", equalTo("json_schema")) .body("schema.schema", equalTo("{\"type\": \"object\", \"properties\": {\"foo\": " + "{\"type\": \"string\"}, \"bar\": {\"type\": \"object\", \"properties\": " + diff --git a/src/main/resources/application.yml b/src/main/resources/application.yml index 56749103e6..4bb7797e2a 100644 --- a/src/main/resources/application.yml +++ b/src/main/resources/application.yml @@ -105,7 +105,7 @@ nakadi: timelines: storage: default: "default" - compacted: "compacted" + compacted: "default" kpi: config: batch-collection-timeout: 1000 From d52cae017437b3cff56d5872a51218aae0293e6b Mon Sep 17 00:00:00 2001 From: vstepanov Date: Fri, 6 Jul 2018 17:08:45 +0200 Subject: [PATCH 05/26] ARUHA-1757: added restrictions to log compaction functionality; --- .../controller/TimelinesController.java | 16 +++++++----- .../TimelinesNotSupportedException.java | 9 +++++++ .../nakadi/service/EventTypeService.java | 26 +++++++++++++++++++ .../service/timeline/TimelineService.java | 5 ++++ 4 files changed, 50 insertions(+), 6 deletions(-) create mode 100644 src/main/java/org/zalando/nakadi/exceptions/runtime/TimelinesNotSupportedException.java diff --git a/src/main/java/org/zalando/nakadi/controller/TimelinesController.java b/src/main/java/org/zalando/nakadi/controller/TimelinesController.java index a33a362449..ad38bcd153 100644 --- a/src/main/java/org/zalando/nakadi/controller/TimelinesController.java +++ b/src/main/java/org/zalando/nakadi/controller/TimelinesController.java @@ -12,14 +12,16 @@ import org.springframework.web.bind.annotation.RequestMethod; import org.springframework.web.bind.annotation.RestController; import org.springframework.web.context.request.NativeWebRequest; -import org.zalando.nakadi.exceptions.runtime.ConflictException; -import org.zalando.nakadi.exceptions.runtime.NotFoundException; -import org.zalando.nakadi.exceptions.runtime.TimelineException; -import org.zalando.nakadi.exceptions.runtime.UnableProcessException; import org.zalando.nakadi.exceptions.runtime.AccessDeniedException; +import org.zalando.nakadi.exceptions.runtime.ConflictException; import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; +import org.zalando.nakadi.exceptions.runtime.MyNakadiRuntimeException1; +import org.zalando.nakadi.exceptions.runtime.NotFoundException; import org.zalando.nakadi.exceptions.runtime.RepositoryProblemException; +import org.zalando.nakadi.exceptions.runtime.TimelineException; +import org.zalando.nakadi.exceptions.runtime.TimelinesNotSupportedException; import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; +import org.zalando.nakadi.exceptions.runtime.UnableProcessException; import org.zalando.nakadi.service.timeline.TimelineService; import org.zalando.nakadi.view.TimelineRequest; import org.zalando.nakadi.view.TimelineView; @@ -66,8 +68,10 @@ public ResponseEntity unprocessable(final UnableProcessException ex, fi return Responses.create(MoreStatus.UNPROCESSABLE_ENTITY, ex.getMessage(), request); } - @ExceptionHandler(NotFoundException.class) - public ResponseEntity notFound(final NotFoundException ex, final NativeWebRequest request) { + @ExceptionHandler({ + NotFoundException.class, + TimelinesNotSupportedException.class}) + public ResponseEntity notFound(final MyNakadiRuntimeException1 ex, final NativeWebRequest request) { LOG.error(ex.getMessage(), ex); return Responses.create(Response.Status.NOT_FOUND, ex.getMessage(), request); } diff --git a/src/main/java/org/zalando/nakadi/exceptions/runtime/TimelinesNotSupportedException.java b/src/main/java/org/zalando/nakadi/exceptions/runtime/TimelinesNotSupportedException.java new file mode 100644 index 0000000000..5b4a8d08ff --- /dev/null +++ b/src/main/java/org/zalando/nakadi/exceptions/runtime/TimelinesNotSupportedException.java @@ -0,0 +1,9 @@ +package org.zalando.nakadi.exceptions.runtime; + +public class TimelinesNotSupportedException extends MyNakadiRuntimeException1 { + + public TimelinesNotSupportedException(final String message) { + super(message); + } + +} diff --git a/src/main/java/org/zalando/nakadi/service/EventTypeService.java b/src/main/java/org/zalando/nakadi/service/EventTypeService.java index 9037e126b8..82bedbbe06 100644 --- a/src/main/java/org/zalando/nakadi/service/EventTypeService.java +++ b/src/main/java/org/zalando/nakadi/service/EventTypeService.java @@ -15,6 +15,7 @@ import org.springframework.transaction.TransactionException; import org.springframework.transaction.support.TransactionTemplate; import org.zalando.nakadi.config.NakadiSettings; +import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.CompatibilityMode; import org.zalando.nakadi.domain.EventCategory; import org.zalando.nakadi.domain.EventType; @@ -148,6 +149,7 @@ public void create(final EventTypeBase eventType) eventTypeOptionsValidator.checkRetentionTime(eventType.getOptions()); setDefaultEventTypeOptions(eventType); validateSchema(eventType); + validateCompaction(eventType); enrichment.validate(eventType); partitionResolver.validate(eventType); authorizationValidator.validateAuthorization(eventType.getAuthorization()); @@ -173,6 +175,19 @@ public void create(final EventTypeBase eventType) .put("compatibility_mode", eventType.getCompatibilityMode())); } + private void validateCompaction(final EventTypeBase eventType) { + if (eventType.getCleanupPolicy() != CleanupPolicy.COMPACT && + !eventType.getPartitionCompactionKeys().isEmpty()) { + throw new InvalidEventTypeException( + "partition_compaction_keys can be only defined when cleanup_policy is 'compact'"); + } + if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT && + eventType.getPartitionCompactionKeys().isEmpty()) { + throw new InvalidEventTypeException( + "partition_compaction_keys should be defined when using cleanup_policy 'compact'"); + } + } + private String identifyAuthzState(final EventTypeBase eventType) { if (eventType.getAuthorization() == null) { return "disabled"; @@ -486,6 +501,7 @@ private void validateSchema(final EventTypeBase eventType) throws InvalidEventTy } validateOrderingKeys(schema, eventType); + validatePartitionCompactionKeys(schema, eventType); if (eventType.getCompatibilityMode() == CompatibilityMode.COMPATIBLE) { validateJsonSchemaConstraints(schemaAsJson); @@ -527,6 +543,16 @@ private void validateOrderingKeys(final Schema schema, final EventTypeBase event } } + private void validatePartitionCompactionKeys(final Schema schema, final EventTypeBase eventType) + throws InvalidEventTypeException, JSONException, SchemaException { + final List absentFields = eventType.getPartitionCompactionKeys().stream() + .filter(field -> !schema.definesProperty(convertToJSONPointer(field))) + .collect(Collectors.toList()); + if (!absentFields.isEmpty()) { + throw new InvalidEventTypeException("partition_compaction_keys " + absentFields + " absent in schema"); + } + } + private String convertToJSONPointer(final String value) { return value.replaceAll("\\.", "/"); } diff --git a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java index cd32a523d2..fe078e28aa 100644 --- a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java +++ b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java @@ -33,6 +33,7 @@ import org.zalando.nakadi.exceptions.runtime.RepositoryProblemException; import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; import org.zalando.nakadi.exceptions.runtime.TimelineException; +import org.zalando.nakadi.exceptions.runtime.TimelinesNotSupportedException; import org.zalando.nakadi.exceptions.runtime.TopicCreationException; import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; @@ -113,6 +114,10 @@ public void createTimeline(final String eventTypeName, final String storageId) final Resource resource = new EventTypeResource(eventTypeName, eventType.getAuthorization()); throw new AccessDeniedException(AuthorizationService.Operation.ADMIN, resource); } + if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { + throw new TimelinesNotSupportedException("It is not possible to create a timeline " + + "for event type with 'compact' cleanup_policy"); + } final Storage storage = storageDbRepository.getStorage(storageId) .orElseThrow(() -> new UnableProcessException("No storage with id: " + storageId)); From 02907a254fcf196bad162e742420e114a0297cd4 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Sat, 7 Jul 2018 09:10:07 +0200 Subject: [PATCH 06/26] ARUHA-1757: added tests; --- .../nakadi/webservice/EventTypeAT.java | 28 +++++++++- .../nakadi/service/EventPublisher.java | 5 +- .../nakadi/service/EventPublisherTest.java | 2 +- .../service/timeline/TimelineServiceTest.java | 52 ++++++++++++------- 4 files changed, 61 insertions(+), 26 deletions(-) diff --git a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java index 66de38cf93..a75a2b722e 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java @@ -3,6 +3,10 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.collect.ImmutableList; import org.apache.http.HttpStatus; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; import org.joda.time.DateTime; import org.json.JSONObject; import org.junit.Assert; @@ -222,8 +226,9 @@ public void whenUpdateRetentionTimeThenUpdateInKafkaAndDB() throws Exception { assertRetentionTime(newRetentionTime, eventType.getName()); } - @Test - public void whenPostCompactedEventTypeThenOk() throws JsonProcessingException { + @Test(timeout = 10000) + public void whenPostCompactedEventTypeThenOk() throws IOException { + // create event type with 'compact' cleanup_policy final EventType eventType = buildDefaultEventType(); eventType.setCleanupPolicy(CleanupPolicy.COMPACT); eventType.setPartitionCompactionKeys(ImmutableList.of("key1")); @@ -236,6 +241,7 @@ public void whenPostCompactedEventTypeThenOk() throws JsonProcessingException { .then() .statusCode(HttpStatus.SC_CREATED); + // get event type and check that properties are set correctly given().body(body) .contentType(JSON) .get(ENDPOINT + "/" + eventType.getName()) @@ -243,6 +249,24 @@ public void whenPostCompactedEventTypeThenOk() throws JsonProcessingException { .statusCode(HttpStatus.SC_OK) .body("cleanup_policy", equalTo("compact")) .body("partition_compaction_keys", equalTo(ImmutableList.of("key1"))); + + // assert that created topic in kafka has correct cleanup_policy + final String topic = (String) NakadiTestUtils.listTimelines(eventType.getName()).get(0).get("topic"); + final String cleanupPolicy = KafkaTestHelper.getTopicCleanupPolicy(topic, ZOOKEEPER_URL); + assertThat(cleanupPolicy, equalTo("compact")); + + // publish event to compacted event type + publishEvent(eventType.getName(), "{\"key1\":\"v1\"}"); + + // assert that key was correctly propagated to event key in kafka + final KafkaTestHelper kafkaHelper = new KafkaTestHelper(KAFKA_URL); + final KafkaConsumer consumer = kafkaHelper.createConsumer(); + final TopicPartition tp = new TopicPartition(topic, 0); + consumer.assign(ImmutableList.of(tp)); + consumer.seek(tp, 0); + final ConsumerRecords records = consumer.poll(5000); + final ConsumerRecord record = records.iterator().next(); + assertThat(record.key(), equalTo("[\"v1\"]")); } @Test diff --git a/src/main/java/org/zalando/nakadi/service/EventPublisher.java b/src/main/java/org/zalando/nakadi/service/EventPublisher.java index 25572fc465..557e834554 100644 --- a/src/main/java/org/zalando/nakadi/service/EventPublisher.java +++ b/src/main/java/org/zalando/nakadi/service/EventPublisher.java @@ -59,7 +59,6 @@ public class EventPublisher { private final Enrichment enrichment; private final TimelineSync timelineSync; private final AuthorizationValidator authValidator; - private final FeatureToggleService featureToggleService; @Autowired public EventPublisher(final TimelineService timelineService, @@ -68,8 +67,7 @@ public EventPublisher(final TimelineService timelineService, final Enrichment enrichment, final NakadiSettings nakadiSettings, final TimelineSync timelineSync, - final AuthorizationValidator authValidator, - final FeatureToggleService featureToggleService) { + final AuthorizationValidator authValidator) { this.timelineService = timelineService; this.eventTypeCache = eventTypeCache; this.partitionResolver = partitionResolver; @@ -77,7 +75,6 @@ public EventPublisher(final TimelineService timelineService, this.nakadiSettings = nakadiSettings; this.timelineSync = timelineSync; this.authValidator = authValidator; - this.featureToggleService = featureToggleService; } public EventPublishResult publish(final String events, final String eventTypeName) diff --git a/src/test/java/org/zalando/nakadi/service/EventPublisherTest.java b/src/test/java/org/zalando/nakadi/service/EventPublisherTest.java index 161178c72d..404a7c3113 100644 --- a/src/test/java/org/zalando/nakadi/service/EventPublisherTest.java +++ b/src/test/java/org/zalando/nakadi/service/EventPublisherTest.java @@ -82,7 +82,7 @@ public EventPublisherTest() { Mockito.when(ts.getActiveTimeline(any(EventType.class))).thenReturn(timeline); publisher = new EventPublisher(ts, cache, partitionResolver, enrichment, nakadiSettings, timelineSync, - authzValidator, Mockito.mock(FeatureToggleService.class)); + authzValidator); } @Test diff --git a/src/test/java/org/zalando/nakadi/service/timeline/TimelineServiceTest.java b/src/test/java/org/zalando/nakadi/service/timeline/TimelineServiceTest.java index d145704682..27153642cc 100644 --- a/src/test/java/org/zalando/nakadi/service/timeline/TimelineServiceTest.java +++ b/src/test/java/org/zalando/nakadi/service/timeline/TimelineServiceTest.java @@ -7,6 +7,7 @@ import org.springframework.transaction.PlatformTransactionManager; import org.springframework.transaction.support.TransactionTemplate; import org.zalando.nakadi.config.NakadiSettings; +import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.DefaultStorage; import org.zalando.nakadi.domain.EventType; import org.zalando.nakadi.domain.Storage; @@ -16,6 +17,7 @@ import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; import org.zalando.nakadi.exceptions.runtime.NotFoundException; import org.zalando.nakadi.exceptions.runtime.TimelineException; +import org.zalando.nakadi.exceptions.runtime.TimelinesNotSupportedException; import org.zalando.nakadi.repository.TopicRepository; import org.zalando.nakadi.repository.TopicRepositoryHolder; import org.zalando.nakadi.repository.db.EventTypeCache; @@ -34,6 +36,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.zalando.nakadi.utils.TestUtils.buildDefaultEventType; public class TimelineServiceTest { @@ -51,16 +54,16 @@ topicRepositoryHolder, new TransactionTemplate(mock(PlatformTransactionManager.c @Test(expected = NotFoundException.class) public void testGetTimelinesNotFound() throws Exception { - Mockito.when(adminService.isAdmin(any())).thenReturn(true); - Mockito.when(eventTypeCache.getEventType(any())).thenThrow(new NoSuchEventTypeException("")); + when(adminService.isAdmin(any())).thenReturn(true); + when(eventTypeCache.getEventType(any())).thenThrow(new NoSuchEventTypeException("")); timelineService.getTimelines("event_type"); } @Test(expected = TimelineException.class) public void testGetTimelinesException() throws Exception { - Mockito.when(adminService.isAdmin(any())).thenReturn(true); - Mockito.when(eventTypeCache.getEventType(any())).thenThrow(new InternalNakadiException("")); + when(adminService.isAdmin(any())).thenReturn(true); + when(eventTypeCache.getEventType(any())).thenThrow(new InternalNakadiException("")); timelineService.getTimelines("event_type"); } @@ -70,7 +73,7 @@ public void testGetTimeline() throws Exception { final EventType eventType = EventTypeTestBuilder.builder().build(); final Timeline timeline = Timeline.createTimeline(eventType.getName(), 0, null, "topic", new Date()); timeline.setSwitchedAt(new Date()); - Mockito.when(eventTypeCache.getTimelinesOrdered(eventType.getName())) + when(eventTypeCache.getTimelinesOrdered(eventType.getName())) .thenReturn(Collections.singletonList(timeline)); final Timeline actualTimeline = timelineService.getActiveTimeline(eventType); @@ -85,22 +88,22 @@ public void testGetActiveTimelinesOrderedFilters() throws Exception { .mapToObj(x -> mock(Timeline.class)) .collect(Collectors.toList()); - Mockito.when(testTimelines.get(0).getSwitchedAt()).thenReturn(new Date()); - Mockito.when(testTimelines.get(0).isDeleted()).thenReturn(false); + when(testTimelines.get(0).getSwitchedAt()).thenReturn(new Date()); + when(testTimelines.get(0).isDeleted()).thenReturn(false); - Mockito.when(testTimelines.get(1).getSwitchedAt()).thenReturn(new Date()); - Mockito.when(testTimelines.get(1).isDeleted()).thenReturn(false); + when(testTimelines.get(1).getSwitchedAt()).thenReturn(new Date()); + when(testTimelines.get(1).isDeleted()).thenReturn(false); - Mockito.when(testTimelines.get(2).getSwitchedAt()).thenReturn(null); - Mockito.when(testTimelines.get(2).isDeleted()).thenReturn(false); + when(testTimelines.get(2).getSwitchedAt()).thenReturn(null); + when(testTimelines.get(2).isDeleted()).thenReturn(false); - Mockito.when(testTimelines.get(3).getSwitchedAt()).thenReturn(new Date()); - Mockito.when(testTimelines.get(3).isDeleted()).thenReturn(true); + when(testTimelines.get(3).getSwitchedAt()).thenReturn(new Date()); + when(testTimelines.get(3).isDeleted()).thenReturn(true); - Mockito.when(testTimelines.get(4).getSwitchedAt()).thenReturn(new Date()); - Mockito.when(testTimelines.get(4).isDeleted()).thenReturn(false); + when(testTimelines.get(4).getSwitchedAt()).thenReturn(new Date()); + when(testTimelines.get(4).isDeleted()).thenReturn(false); - Mockito.when(eventTypeCache.getTimelinesOrdered(eq(eventTypeName))).thenReturn(testTimelines); + when(eventTypeCache.getTimelinesOrdered(eq(eventTypeName))).thenReturn(testTimelines); final List expectedResult = ImmutableList.of(testTimelines.get(0), testTimelines.get(1), testTimelines.get(4)); @@ -111,8 +114,8 @@ public void testGetActiveTimelinesOrderedFilters() throws Exception { @Test public void shouldDeleteTopicWhenTimelineCreationFails() throws Exception { final TopicRepository repository = mock(TopicRepository.class); - Mockito.when(topicRepositoryHolder.getTopicRepository(any())).thenReturn(repository); - Mockito.when(timelineDbRepository.createTimeline(any())) + when(topicRepositoryHolder.getTopicRepository(any())).thenReturn(repository); + when(timelineDbRepository.createTimeline(any())) .thenThrow(new InconsistentStateException("shouldDeleteTopicWhenTimelineCreationFails")); try { timelineService.createDefaultTimeline(buildDefaultEventType(), 1); @@ -130,7 +133,7 @@ public void shouldDeleteAllTimelinesWhenOneTimelineWasMarkedAsDeleted() throws E t1.setSwitchedAt(new Date()); final Timeline t2 = Timeline.createTimeline(eventType.getName(), 2, null, "topic2", new Date()); t2.setSwitchedAt(new Date()); - Mockito.when(eventTypeCache.getTimelinesOrdered(eventType.getName())) + when(eventTypeCache.getTimelinesOrdered(eventType.getName())) .thenReturn(ImmutableList.of(t1, t2)); timelineService.deleteAllTimelinesForEventType(eventType.getName()); @@ -138,4 +141,15 @@ public void shouldDeleteAllTimelinesWhenOneTimelineWasMarkedAsDeleted() throws E Mockito.verify(timelineDbRepository, Mockito.times(2)).deleteTimeline(Mockito.any()); } + @Test(expected = TimelinesNotSupportedException.class) + public void whenCreateTimelineForCompactedEventTypeThenException() throws Exception { + final EventType eventType = buildDefaultEventType(); + eventType.setCleanupPolicy(CleanupPolicy.COMPACT); + when(eventTypeCache.getEventType("et1")).thenReturn(eventType); + + when(adminService.isAdmin(any())).thenReturn(true); + + timelineService.createTimeline("et1", "st1"); + } + } From ae10771fa59a84074d3605856d4334510b73bd17 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Sat, 7 Jul 2018 09:24:06 +0200 Subject: [PATCH 07/26] ARUHA-1757: modified test; --- .../repository/kafka/KafkaRepositoryAT.java | 23 +++---------------- 1 file changed, 3 insertions(+), 20 deletions(-) diff --git a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java index cd5aee2b5c..ef0a03feec 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java @@ -36,7 +36,6 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import static org.zalando.nakadi.utils.TestUtils.waitFor; public class KafkaRepositoryAT extends BaseAT { @@ -111,6 +110,9 @@ public void whenCreateTopicThenTopicIsCreated() throws Exception { partitionInfos.forEach(pInfo -> assertThat(pInfo.replicas(), arrayWithSize(DEFAULT_REPLICA_FACTOR))); + + final String cleanupPolicy = KafkaTestHelper.getTopicCleanupPolicy(topicName, ZOOKEEPER_URL); + assertThat(cleanupPolicy, equalTo("delete")); }, new RetryForSpecifiedTimeStrategy(5000).withExceptionsThatForceRetry(AssertionError.class) .withWaitBetweenEachTry(500)); @@ -177,25 +179,6 @@ public void whenCreateTopicWithRetentionTime() throws Exception { .withWaitBetweenEachTry(500)); } - @Test(timeout = 10000) - @SuppressWarnings("unchecked") - public void checkCompactionCleanupPolicySetCorrectly() { - setCleanupPolicyAndCheck(CleanupPolicy.DELETE, "delete"); - setCleanupPolicyAndCheck(CleanupPolicy.COMPACT, "compact"); - } - - private void setCleanupPolicyAndCheck(final CleanupPolicy cleanupPolicy, final String expectedCleanupPolicy) { - // ACT // - final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, DEFAULT_RETENTION_TIME, - cleanupPolicy); - - // ASSERT // - waitFor(() -> { - final String actualCleanupPolicy = KafkaTestHelper.getTopicCleanupPolicy(topicName, ZOOKEEPER_URL); - assertThat(actualCleanupPolicy, equalTo(expectedCleanupPolicy)); - }, 5000); - } - private Map> getAllTopics() { final KafkaConsumer kafkaConsumer = kafkaHelper.createConsumer(); return kafkaConsumer.listTopics(); From c8819313cd2bae803d76cd41a40bb998a9263ed5 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Sun, 8 Jul 2018 22:48:00 +0200 Subject: [PATCH 08/26] ARUHA-1757: added update restrictions; extended tests; --- .../nakadi/webservice/EventTypeAT.java | 15 ++-- .../zalando/nakadi/domain/EventTypeBase.java | 4 +- .../nakadi/service/EventTypeService.java | 13 +++- src/main/resources/application.yml | 7 +- .../controller/EventTypeControllerTest.java | 73 +++++++++++++++++++ .../nakadi/utils/EventTypeTestBuilder.java | 8 +- 6 files changed, 107 insertions(+), 13 deletions(-) diff --git a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java index a75a2b722e..3b16d1af74 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java @@ -227,12 +227,15 @@ public void whenUpdateRetentionTimeThenUpdateInKafkaAndDB() throws Exception { } @Test(timeout = 10000) - public void whenPostCompactedEventTypeThenOk() throws IOException { + public void compactedEventTypeJourney() throws IOException { // create event type with 'compact' cleanup_policy final EventType eventType = buildDefaultEventType(); eventType.setCleanupPolicy(CleanupPolicy.COMPACT); - eventType.setPartitionCompactionKeys(ImmutableList.of("key1")); - eventType.getSchema().setSchema("{\"type\":\"object\",\"properties\":{\"key1\":{\"type\":\"string\"}}}"); + eventType.setPartitionCompactionKeys(ImmutableList.of("key1", "some_field.key2")); + eventType.getSchema().setSchema( + "{\"type\":\"object\",\"properties\":{" + + "\"key1\":{\"type\":\"string\"}," + + "\"some_field\":{\"type\":\"object\",\"properties\":{\"key2\":{\"type\":\"integer\"}}}}}"); final String body = MAPPER.writer().writeValueAsString(eventType); given().body(body) @@ -248,7 +251,7 @@ public void whenPostCompactedEventTypeThenOk() throws IOException { .then() .statusCode(HttpStatus.SC_OK) .body("cleanup_policy", equalTo("compact")) - .body("partition_compaction_keys", equalTo(ImmutableList.of("key1"))); + .body("partition_compaction_keys", equalTo(ImmutableList.of("key1", "some_field.key2"))); // assert that created topic in kafka has correct cleanup_policy final String topic = (String) NakadiTestUtils.listTimelines(eventType.getName()).get(0).get("topic"); @@ -256,7 +259,7 @@ public void whenPostCompactedEventTypeThenOk() throws IOException { assertThat(cleanupPolicy, equalTo("compact")); // publish event to compacted event type - publishEvent(eventType.getName(), "{\"key1\":\"v1\"}"); + publishEvent(eventType.getName(), "{\"key1\":\"v1\",\"some_field\":{\"key2\":2}}"); // assert that key was correctly propagated to event key in kafka final KafkaTestHelper kafkaHelper = new KafkaTestHelper(KAFKA_URL); @@ -266,7 +269,7 @@ public void whenPostCompactedEventTypeThenOk() throws IOException { consumer.seek(tp, 0); final ConsumerRecords records = consumer.poll(5000); final ConsumerRecord record = records.iterator().next(); - assertThat(record.key(), equalTo("[\"v1\"]")); + assertThat(record.key(), equalTo("[\"v1\",\"2\"]")); } @Test diff --git a/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java b/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java index d74319ebed..54534a529d 100644 --- a/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java +++ b/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java @@ -91,7 +91,8 @@ public EventTypeBase(final String name, final String owningApplication, final EventTypeStatistics defaultStatistic, final EventTypeOptions options, final CompatibilityMode compatibilityMode, - final CleanupPolicy cleanupPolicy) { + final CleanupPolicy cleanupPolicy, + final List partitionCompactionKeys) { this.name = name; this.owningApplication = owningApplication; this.category = category; @@ -104,6 +105,7 @@ public EventTypeBase(final String name, final String owningApplication, this.options = options; this.compatibilityMode = compatibilityMode; this.cleanupPolicy = cleanupPolicy; + this.partitionCompactionKeys = partitionCompactionKeys; } public EventTypeBase(final EventTypeBase eventType) { diff --git a/src/main/java/org/zalando/nakadi/service/EventTypeService.java b/src/main/java/org/zalando/nakadi/service/EventTypeService.java index 82bedbbe06..553936bf10 100644 --- a/src/main/java/org/zalando/nakadi/service/EventTypeService.java +++ b/src/main/java/org/zalando/nakadi/service/EventTypeService.java @@ -148,8 +148,8 @@ public void create(final EventTypeBase eventType) } eventTypeOptionsValidator.checkRetentionTime(eventType.getOptions()); setDefaultEventTypeOptions(eventType); - validateSchema(eventType); validateCompaction(eventType); + validateSchema(eventType); enrichment.validate(eventType); partitionResolver.validate(eventType); authorizationValidator.validateAuthorization(eventType.getAuthorization()); @@ -188,6 +188,16 @@ private void validateCompaction(final EventTypeBase eventType) { } } + private void validateCompactionUpdate(final EventType original, final EventTypeBase updatedET) { + validateCompaction(updatedET); + if (original.getCleanupPolicy() != updatedET.getCleanupPolicy()) { + throw new InvalidEventTypeException("cleanup_policy can not be changed"); + } + if (!original.getPartitionCompactionKeys().equals(updatedET.getPartitionCompactionKeys())) { + throw new InvalidEventTypeException("partition_compaction_keys can not be changed"); + } + } + private String identifyAuthzState(final EventTypeBase eventType) { if (eventType.getAuthorization() == null) { return "disabled"; @@ -331,6 +341,7 @@ public void update(final String eventTypeName, } authorizationValidator.validateAuthorization(original, eventTypeBase); validateName(eventTypeName, eventTypeBase); + validateCompactionUpdate(original, eventTypeBase); validateSchema(eventTypeBase); validateAudience(original, eventTypeBase); partitionResolver.validate(eventTypeBase); diff --git a/src/main/resources/application.yml b/src/main/resources/application.yml index 4bb7797e2a..f0a386caca 100644 --- a/src/main/resources/application.yml +++ b/src/main/resources/application.yml @@ -102,10 +102,9 @@ nakadi: request.timeout: 2000 connect.timeout: 1000 socket.timeout: 2000 - timelines: - storage: - default: "default" - compacted: "default" + timelines.storage: + default: "default" + compacted: "default" kpi: config: batch-collection-timeout: 1000 diff --git a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java index 329787e1a0..4e95514af4 100644 --- a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java +++ b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java @@ -15,6 +15,7 @@ import org.springframework.test.web.servlet.request.MockHttpServletRequestBuilder; import org.zalando.nakadi.config.SecuritySettings; import org.zalando.nakadi.domain.Audience; +import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.EnrichmentStrategyDescriptor; import org.zalando.nakadi.domain.EventType; import org.zalando.nakadi.domain.EventTypeBase; @@ -233,6 +234,78 @@ public void whenPUTwithNullAudienceThen422() throws Exception { .andExpect(status().isUnprocessableEntity()); } + @Test + public void whenPostCompactedETWithoutCompactionKeysThen422() throws Exception { + final EventType eventType = EventTypeTestBuilder.builder() + .cleanupPolicy(CleanupPolicy.COMPACT) + .build(); + postEventType(eventType).andExpect(status().isUnprocessableEntity()); + } + + @Test + public void whenCompactionKeysSpecifiedForNoneCompactedETThen422() throws Exception { + final EventType eventType = EventTypeTestBuilder.builder() + .partitionCompactionKeys(ImmutableList.of("key")) + .schema("{\"type\":\"object\",\"properties\":{\"key\":{\"type\":\"string\"}}}") + .build(); + postEventType(eventType).andExpect(status().isUnprocessableEntity()); + } + + @Test + public void whenCompactionKeysNotSpecifiedInSchemaThen422() throws Exception { + final EventType eventType = EventTypeTestBuilder.builder() + .cleanupPolicy(CleanupPolicy.COMPACT) + .partitionCompactionKeys(ImmutableList.of("key")) + .build(); + postEventType(eventType).andExpect(status().isUnprocessableEntity()); + } + + @Test + public void whenPutEventTypeWithChangedCleanupPolicyThen422() throws Exception { + final EventType originalEventType = EventTypeTestBuilder.builder() + .cleanupPolicy(CleanupPolicy.COMPACT) + .partitionCompactionKeys(ImmutableList.of("key")) + .schema("{\"type\":\"object\",\"properties\":{\"key\":{\"type\":\"string\"}}}") + .build(); + + final EventType updatedEventType = EventTypeTestBuilder.builder() + .name(originalEventType.getName()) + .cleanupPolicy(CleanupPolicy.DELETE) + .partitionCompactionKeys(ImmutableList.of("key")) + .schema("{\"type\":\"object\",\"properties\":{\"key\":{\"type\":\"string\"}}}") + .build(); + + doReturn(originalEventType).when(eventTypeRepository).findByName(any()); + + putEventType(updatedEventType, originalEventType.getName()) + .andExpect(status().isUnprocessableEntity()); + } + + @Test + public void whenPutEventTypeWithChangedCompactionKeysThen422() throws Exception { + final EventType originalEventType = EventTypeTestBuilder.builder() + .cleanupPolicy(CleanupPolicy.COMPACT) + .partitionCompactionKeys(ImmutableList.of("key")) + .schema("{\"type\":\"object\",\"properties\":{" + + "\"key\":{\"type\":\"string\"}," + + "\"key2\":{\"type\":\"string\"}}}") + .build(); + + final EventType updatedEventType = EventTypeTestBuilder.builder() + .name(originalEventType.getName()) + .cleanupPolicy(CleanupPolicy.COMPACT) + .partitionCompactionKeys(ImmutableList.of("key2")) + .schema("{\"type\":\"object\",\"properties\":{" + + "\"key\":{\"type\":\"string\"}," + + "\"key2\":{\"type\":\"string\"}}}") + .build(); + + doReturn(originalEventType).when(eventTypeRepository).findByName(any()); + + putEventType(updatedEventType, originalEventType.getName()) + .andExpect(status().isUnprocessableEntity()); + } + @Test public void whenPUTthenWarning() throws Exception { final EventType eventType = buildDefaultEventType(); diff --git a/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java b/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java index f3f177bd30..b3010bc1c4 100644 --- a/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java +++ b/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java @@ -39,6 +39,7 @@ public class EventTypeTestBuilder { private EventTypeOptions options; private CompatibilityMode compatibilityMode; private CleanupPolicy cleanupPolicy; + private List partitionCompactionKeys; private DateTime createdAt; private DateTime updatedAt; private Audience audience; @@ -136,6 +137,11 @@ public EventTypeTestBuilder cleanupPolicy(final CleanupPolicy cleanupPolicy) { return this; } + public EventTypeTestBuilder partitionCompactionKeys(final List partitionCompactionKeys) { + this.partitionCompactionKeys = partitionCompactionKeys; + return this; + } + public EventTypeTestBuilder createdAt(final DateTime createdAt) { this.createdAt = createdAt; return this; @@ -159,7 +165,7 @@ public EventTypeTestBuilder audience(final Audience audience) { public EventType build() { final EventTypeBase eventTypeBase = new EventTypeBase(name, owningApplication, category, validationStrategies, enrichmentStrategies, partitionStrategy, partitionKeyFields, schema, - defaultStatistic, options, compatibilityMode, cleanupPolicy); + defaultStatistic, options, compatibilityMode, cleanupPolicy, partitionCompactionKeys); eventTypeBase.setAuthorization(authorization); eventTypeBase.setAudience(audience); return new EventType(eventTypeBase, this.schema.getVersion().toString(), this.createdAt, this.updatedAt); From 50c8927c1177f76cbca42faa3e6df7e41b48f89a Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 11:36:59 +0200 Subject: [PATCH 09/26] ARUHA-1757: added handling missing compaction key; --- .../nakadi/webservice/EventTypeAT.java | 7 ++++ .../exceptions/CompactionException.java | 9 +++++ .../nakadi/service/EventPublisher.java | 33 ++++++++++--------- 3 files changed, 33 insertions(+), 16 deletions(-) create mode 100644 src/main/java/org/zalando/nakadi/exceptions/CompactionException.java diff --git a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java index 3b16d1af74..115921bc28 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java @@ -270,6 +270,13 @@ public void compactedEventTypeJourney() throws IOException { final ConsumerRecords records = consumer.poll(5000); final ConsumerRecord record = records.iterator().next(); assertThat(record.key(), equalTo("[\"v1\",\"2\"]")); + + // publish event with missing compaction key and expect 422 + given().body("[{\"key1\":\"v1\"}]") + .contentType(JSON) + .post(ENDPOINT + "/" + eventType.getName() + "/events") + .then() + .statusCode(HttpStatus.SC_UNPROCESSABLE_ENTITY); } @Test diff --git a/src/main/java/org/zalando/nakadi/exceptions/CompactionException.java b/src/main/java/org/zalando/nakadi/exceptions/CompactionException.java new file mode 100644 index 0000000000..ae87942474 --- /dev/null +++ b/src/main/java/org/zalando/nakadi/exceptions/CompactionException.java @@ -0,0 +1,9 @@ +package org.zalando.nakadi.exceptions; + +public class CompactionException extends UnprocessableEntityException { + + public CompactionException(final String message) { + super(message); + } + +} diff --git a/src/main/java/org/zalando/nakadi/service/EventPublisher.java b/src/main/java/org/zalando/nakadi/service/EventPublisher.java index 557e834554..64e6449297 100644 --- a/src/main/java/org/zalando/nakadi/service/EventPublisher.java +++ b/src/main/java/org/zalando/nakadi/service/EventPublisher.java @@ -18,6 +18,7 @@ import org.zalando.nakadi.domain.EventType; import org.zalando.nakadi.domain.Timeline; import org.zalando.nakadi.enrichment.Enrichment; +import org.zalando.nakadi.exceptions.CompactionException; import org.zalando.nakadi.exceptions.EnrichmentException; import org.zalando.nakadi.exceptions.InternalNakadiException; import org.zalando.nakadi.exceptions.InvalidPartitionKeyFieldsException; @@ -27,7 +28,6 @@ import org.zalando.nakadi.exceptions.runtime.EventPublishingException; import org.zalando.nakadi.exceptions.runtime.EventTypeTimeoutException; import org.zalando.nakadi.exceptions.runtime.EventValidationException; -import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; import org.zalando.nakadi.partitioning.PartitionResolver; import org.zalando.nakadi.repository.db.EventTypeCache; @@ -44,6 +44,7 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; +import static com.google.common.collect.Lists.newArrayList; import static org.zalando.nakadi.partitioning.HashPartitionStrategy.DATA_PATH_PREFIX; @Component @@ -115,6 +116,9 @@ EventPublishResult publishInternal(final String events, } catch (final PartitioningException e) { LOG.debug("Event partition error: {}", e.getMessage()); return aborted(EventPublishingStep.PARTITIONING, batch); + } catch (final CompactionException e) { + LOG.debug("Event compaction error: {}", e.getMessage()); + return aborted(EventPublishingStep.PARTITIONING, batch); } catch (final EnrichmentException e) { LOG.debug("Event enrichment error: {}", e.getMessage()); return aborted(EventPublishingStep.ENRICHING, batch); @@ -170,24 +174,21 @@ private void partition(final List batch, final EventType eventType) t } } - private void compact(final List batch, final EventType eventType) { + private void compact(final List batch, final EventType eventType) throws CompactionException { if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { for (final BatchItem item : batch) { final JsonPathAccess jsonPath = new JsonPathAccess(item.getEvent()); - final List compactionKeys = eventType.getPartitionCompactionKeys().stream() - .map(compactionKeyField -> EventCategory.DATA.equals(eventType.getCategory()) ? - DATA_PATH_PREFIX + compactionKeyField : compactionKeyField) - .map(compactionKeyField -> { - try { - return jsonPath.get(compactionKeyField).toString(); - } catch (final InvalidPartitionKeyFieldsException e) { - // this should be never thrown as we force users to make compaction keys to be required, - // so if compaction key is missing we should fail earlier on validation step - throw new InconsistentStateException( - "Unexpected exception occurred when assembling compaction key", e); - } - }) - .collect(Collectors.toList()); + final List compactionKeys = newArrayList(); + for (final String compactionKey : eventType.getPartitionCompactionKeys()) { + final String compactionKeyField = EventCategory.DATA.equals(eventType.getCategory()) ? + DATA_PATH_PREFIX + compactionKey : compactionKey; + try { + compactionKeys.add(jsonPath.get(compactionKeyField).toString()); + } catch (final InvalidPartitionKeyFieldsException e) { + item.updateStatusAndDetail(EventPublishingStatus.FAILED, "no compaction key found"); + throw new CompactionException("No compaction key found in event"); + } + } final String compactionKeyStr = new JSONArray(compactionKeys).toString(); item.setEventKey(compactionKeyStr); } From 57f7e44efb700cd1f929e44f66ed7692e037a8fc Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 11:44:49 +0200 Subject: [PATCH 10/26] ARUHA-1757: updated changelog; --- CHANGELOG.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 95be995515..4ab0eac232 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,9 @@ and this project adheres to [Semantic Versioning](http://semver.org/). ## [Unreleased] +### Added +- Log Compaction functionality. + ## [2.7.7] - 2018-06-26 ### Added From f1ffed0879ce8c1deb93df4079bded2c96028c8a Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 13:46:38 +0200 Subject: [PATCH 11/26] ARUHA-1757: moved log compaction key to metadata; --- .../nakadi/webservice/EventTypeAT.java | 29 +++++----- .../zalando/nakadi/domain/EventTypeBase.java | 18 +----- .../nakadi/service/EventPublisher.java | 29 +++------- .../nakadi/service/EventTypeService.java | 30 ---------- .../validation/JsonSchemaEnrichment.java | 34 +++++++----- .../controller/EventTypeControllerTest.java | 55 ------------------- .../nakadi/utils/EventTypeTestBuilder.java | 12 +--- 7 files changed, 48 insertions(+), 159 deletions(-) diff --git a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java index 115921bc28..c5f9a0a449 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/webservice/EventTypeAT.java @@ -12,6 +12,8 @@ import org.junit.Assert; import org.junit.Test; import org.zalando.nakadi.domain.CleanupPolicy; +import org.zalando.nakadi.domain.EnrichmentStrategyDescriptor; +import org.zalando.nakadi.domain.EventCategory; import org.zalando.nakadi.domain.EventType; import org.zalando.nakadi.domain.ResourceAuthorization; import org.zalando.nakadi.domain.ResourceAuthorizationAttribute; @@ -229,14 +231,11 @@ public void whenUpdateRetentionTimeThenUpdateInKafkaAndDB() throws Exception { @Test(timeout = 10000) public void compactedEventTypeJourney() throws IOException { // create event type with 'compact' cleanup_policy - final EventType eventType = buildDefaultEventType(); - eventType.setCleanupPolicy(CleanupPolicy.COMPACT); - eventType.setPartitionCompactionKeys(ImmutableList.of("key1", "some_field.key2")); - eventType.getSchema().setSchema( - "{\"type\":\"object\",\"properties\":{" + - "\"key1\":{\"type\":\"string\"}," + - "\"some_field\":{\"type\":\"object\",\"properties\":{\"key2\":{\"type\":\"integer\"}}}}}"); - + final EventType eventType = EventTypeTestBuilder.builder() + .category(EventCategory.BUSINESS) + .cleanupPolicy(CleanupPolicy.COMPACT) + .enrichmentStrategies(ImmutableList.of(EnrichmentStrategyDescriptor.METADATA_ENRICHMENT)) + .build(); final String body = MAPPER.writer().writeValueAsString(eventType); given().body(body) .contentType(JSON) @@ -250,8 +249,7 @@ public void compactedEventTypeJourney() throws IOException { .get(ENDPOINT + "/" + eventType.getName()) .then() .statusCode(HttpStatus.SC_OK) - .body("cleanup_policy", equalTo("compact")) - .body("partition_compaction_keys", equalTo(ImmutableList.of("key1", "some_field.key2"))); + .body("cleanup_policy", equalTo("compact")); // assert that created topic in kafka has correct cleanup_policy final String topic = (String) NakadiTestUtils.listTimelines(eventType.getName()).get(0).get("topic"); @@ -259,7 +257,10 @@ public void compactedEventTypeJourney() throws IOException { assertThat(cleanupPolicy, equalTo("compact")); // publish event to compacted event type - publishEvent(eventType.getName(), "{\"key1\":\"v1\",\"some_field\":{\"key2\":2}}"); + publishEvent(eventType.getName(), "{\"metadata\":{" + + "\"occurred_at\":\"1992-08-03T10:00:00Z\"," + + "\"eid\":\"329ed3d2-8366-11e8-adc0-fa7ae01bbebc\"," + + "\"partition_compaction_key\":\"abc\"}}"); // assert that key was correctly propagated to event key in kafka final KafkaTestHelper kafkaHelper = new KafkaTestHelper(KAFKA_URL); @@ -269,10 +270,12 @@ public void compactedEventTypeJourney() throws IOException { consumer.seek(tp, 0); final ConsumerRecords records = consumer.poll(5000); final ConsumerRecord record = records.iterator().next(); - assertThat(record.key(), equalTo("[\"v1\",\"2\"]")); + assertThat(record.key(), equalTo("abc")); // publish event with missing compaction key and expect 422 - given().body("[{\"key1\":\"v1\"}]") + given().body("[{\"metadata\":{" + + "\"occurred_at\":\"1992-08-03T10:00:00Z\"," + + "\"eid\":\"329ed3d2-8366-11e8-adc0-fa7ae01bbebc\"}}]") .contentType(JSON) .post(ENDPOINT + "/" + eventType.getName() + "/events") .then() diff --git a/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java b/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java index 54534a529d..029f9d6822 100644 --- a/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java +++ b/src/main/java/org/zalando/nakadi/domain/EventTypeBase.java @@ -19,7 +19,6 @@ public class EventTypeBase { private static final List EMPTY_PARTITION_KEY_FIELDS = ImmutableList.of(); private static final List EMPTY_ORDERING_KEY_FIELDS = ImmutableList.of(); - private static final List EMPTY_PARTITION_COMPACTION_KEYS = ImmutableList.of(); @NotNull @Pattern(regexp = "[a-zA-Z][-0-9a-zA-Z_]*(\\.[0-9a-zA-Z][-0-9a-zA-Z_]*)*", message = "format not allowed") @@ -46,9 +45,6 @@ public class EventTypeBase { @NotNull private CleanupPolicy cleanupPolicy; - @Nullable - private List partitionCompactionKeys; - @Nullable private List orderingKeyFields; @@ -91,8 +87,7 @@ public EventTypeBase(final String name, final String owningApplication, final EventTypeStatistics defaultStatistic, final EventTypeOptions options, final CompatibilityMode compatibilityMode, - final CleanupPolicy cleanupPolicy, - final List partitionCompactionKeys) { + final CleanupPolicy cleanupPolicy) { this.name = name; this.owningApplication = owningApplication; this.category = category; @@ -105,7 +100,6 @@ public EventTypeBase(final String name, final String owningApplication, this.options = options; this.compatibilityMode = compatibilityMode; this.cleanupPolicy = cleanupPolicy; - this.partitionCompactionKeys = partitionCompactionKeys; } public EventTypeBase(final EventTypeBase eventType) { @@ -124,7 +118,6 @@ public EventTypeBase(final EventTypeBase eventType) { this.setAudience(eventType.getAudience()); this.setOrderingKeyFields(eventType.getOrderingKeyFields()); this.setCleanupPolicy(eventType.getCleanupPolicy()); - this.setPartitionCompactionKeys(eventType.getPartitionCompactionKeys()); } public String getName() { @@ -195,15 +188,6 @@ public void setCleanupPolicy(final CleanupPolicy cleanupPolicy) { this.cleanupPolicy = cleanupPolicy; } - public List getPartitionCompactionKeys() { - return unmodifiableList(partitionCompactionKeys != null ? partitionCompactionKeys : - EMPTY_PARTITION_COMPACTION_KEYS); - } - - public void setPartitionCompactionKeys(@Nullable final List partitionCompactionKeys) { - this.partitionCompactionKeys = partitionCompactionKeys; - } - public List getOrderingKeyFields() { return unmodifiableList(orderingKeyFields != null ? orderingKeyFields : EMPTY_ORDERING_KEY_FIELDS); } diff --git a/src/main/java/org/zalando/nakadi/service/EventPublisher.java b/src/main/java/org/zalando/nakadi/service/EventPublisher.java index 64e6449297..372cd9285a 100644 --- a/src/main/java/org/zalando/nakadi/service/EventPublisher.java +++ b/src/main/java/org/zalando/nakadi/service/EventPublisher.java @@ -1,6 +1,6 @@ package org.zalando.nakadi.service; -import org.json.JSONArray; +import org.json.JSONException; import org.json.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -11,7 +11,6 @@ import org.zalando.nakadi.domain.BatchItem; import org.zalando.nakadi.domain.BatchItemResponse; import org.zalando.nakadi.domain.CleanupPolicy; -import org.zalando.nakadi.domain.EventCategory; import org.zalando.nakadi.domain.EventPublishResult; import org.zalando.nakadi.domain.EventPublishingStatus; import org.zalando.nakadi.domain.EventPublishingStep; @@ -21,7 +20,6 @@ import org.zalando.nakadi.exceptions.CompactionException; import org.zalando.nakadi.exceptions.EnrichmentException; import org.zalando.nakadi.exceptions.InternalNakadiException; -import org.zalando.nakadi.exceptions.InvalidPartitionKeyFieldsException; import org.zalando.nakadi.exceptions.NoSuchEventTypeException; import org.zalando.nakadi.exceptions.PartitioningException; import org.zalando.nakadi.exceptions.runtime.AccessDeniedException; @@ -33,7 +31,6 @@ import org.zalando.nakadi.repository.db.EventTypeCache; import org.zalando.nakadi.service.timeline.TimelineService; import org.zalando.nakadi.service.timeline.TimelineSync; -import org.zalando.nakadi.util.JsonPathAccess; import org.zalando.nakadi.validation.EventTypeValidator; import org.zalando.nakadi.validation.ValidationError; @@ -44,9 +41,6 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; -import static com.google.common.collect.Lists.newArrayList; -import static org.zalando.nakadi.partitioning.HashPartitionStrategy.DATA_PATH_PREFIX; - @Component public class EventPublisher { @@ -177,20 +171,15 @@ private void partition(final List batch, final EventType eventType) t private void compact(final List batch, final EventType eventType) throws CompactionException { if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { for (final BatchItem item : batch) { - final JsonPathAccess jsonPath = new JsonPathAccess(item.getEvent()); - final List compactionKeys = newArrayList(); - for (final String compactionKey : eventType.getPartitionCompactionKeys()) { - final String compactionKeyField = EventCategory.DATA.equals(eventType.getCategory()) ? - DATA_PATH_PREFIX + compactionKey : compactionKey; - try { - compactionKeys.add(jsonPath.get(compactionKeyField).toString()); - } catch (final InvalidPartitionKeyFieldsException e) { - item.updateStatusAndDetail(EventPublishingStatus.FAILED, "no compaction key found"); - throw new CompactionException("No compaction key found in event"); - } + try { + final String compactionKey = item.getEvent() + .getJSONObject("metadata") + .getString("partition_compaction_key"); + item.setEventKey(compactionKey); + } catch (JSONException e) { + item.updateStatusAndDetail(EventPublishingStatus.FAILED, "no compaction key found in metadata"); + throw new CompactionException("No compaction key found in event metadata"); } - final String compactionKeyStr = new JSONArray(compactionKeys).toString(); - item.setEventKey(compactionKeyStr); } } } diff --git a/src/main/java/org/zalando/nakadi/service/EventTypeService.java b/src/main/java/org/zalando/nakadi/service/EventTypeService.java index 553936bf10..29938a11a4 100644 --- a/src/main/java/org/zalando/nakadi/service/EventTypeService.java +++ b/src/main/java/org/zalando/nakadi/service/EventTypeService.java @@ -15,7 +15,6 @@ import org.springframework.transaction.TransactionException; import org.springframework.transaction.support.TransactionTemplate; import org.zalando.nakadi.config.NakadiSettings; -import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.CompatibilityMode; import org.zalando.nakadi.domain.EventCategory; import org.zalando.nakadi.domain.EventType; @@ -148,7 +147,6 @@ public void create(final EventTypeBase eventType) } eventTypeOptionsValidator.checkRetentionTime(eventType.getOptions()); setDefaultEventTypeOptions(eventType); - validateCompaction(eventType); validateSchema(eventType); enrichment.validate(eventType); partitionResolver.validate(eventType); @@ -175,27 +173,10 @@ public void create(final EventTypeBase eventType) .put("compatibility_mode", eventType.getCompatibilityMode())); } - private void validateCompaction(final EventTypeBase eventType) { - if (eventType.getCleanupPolicy() != CleanupPolicy.COMPACT && - !eventType.getPartitionCompactionKeys().isEmpty()) { - throw new InvalidEventTypeException( - "partition_compaction_keys can be only defined when cleanup_policy is 'compact'"); - } - if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT && - eventType.getPartitionCompactionKeys().isEmpty()) { - throw new InvalidEventTypeException( - "partition_compaction_keys should be defined when using cleanup_policy 'compact'"); - } - } - private void validateCompactionUpdate(final EventType original, final EventTypeBase updatedET) { - validateCompaction(updatedET); if (original.getCleanupPolicy() != updatedET.getCleanupPolicy()) { throw new InvalidEventTypeException("cleanup_policy can not be changed"); } - if (!original.getPartitionCompactionKeys().equals(updatedET.getPartitionCompactionKeys())) { - throw new InvalidEventTypeException("partition_compaction_keys can not be changed"); - } } private String identifyAuthzState(final EventTypeBase eventType) { @@ -512,7 +493,6 @@ private void validateSchema(final EventTypeBase eventType) throws InvalidEventTy } validateOrderingKeys(schema, eventType); - validatePartitionCompactionKeys(schema, eventType); if (eventType.getCompatibilityMode() == CompatibilityMode.COMPATIBLE) { validateJsonSchemaConstraints(schemaAsJson); @@ -554,16 +534,6 @@ private void validateOrderingKeys(final Schema schema, final EventTypeBase event } } - private void validatePartitionCompactionKeys(final Schema schema, final EventTypeBase eventType) - throws InvalidEventTypeException, JSONException, SchemaException { - final List absentFields = eventType.getPartitionCompactionKeys().stream() - .filter(field -> !schema.definesProperty(convertToJSONPointer(field))) - .collect(Collectors.toList()); - if (!absentFields.isEmpty()) { - throw new InvalidEventTypeException("partition_compaction_keys " + absentFields + " absent in schema"); - } - } - private String convertToJSONPointer(final String value) { return value.replaceAll("\\.", "/"); } diff --git a/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java b/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java index c3df452f65..bc05b4e2d0 100644 --- a/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java +++ b/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java @@ -34,9 +34,12 @@ public JSONObject effectiveSchema(final EventType eventType) throws JSONExceptio } switch (eventType.getCategory()) { - case BUSINESS: return addMetadata(schema, eventType); - case DATA: return wrapSchemaInData(schema, eventType); - default: return schema; + case BUSINESS: + return addMetadata(schema, eventType); + case DATA: + return wrapSchemaInData(schema, eventType); + default: + return schema; } } @@ -47,7 +50,7 @@ private void enforceStrictValidation(final JSONObject schema) { COMPOSED_SCHEMA_KEYWORDS.forEach(keyword -> { if (schema.has(keyword)) { schema.getJSONArray(keyword) - .forEach(object -> enforceStrictValidation((JSONObject)object)); + .forEach(object -> enforceStrictValidation((JSONObject) object)); } }); } @@ -95,16 +98,16 @@ private void enforceNoAdditionalProperties(final JSONObject schema) { .ifPresent(object -> object.keySet().forEach(key -> enforceStrictValidation(object.getJSONObject(key))) ); - }); + }); } private boolean isEmptySchema(final JSONObject schema) { return !( - OBJECT_SCHEMA_KEYWORDS.stream().anyMatch(schema::has) || - ARRAY_SCHEMA_KEYWORDS.stream().anyMatch(schema::has) || - COMPOSED_SCHEMA_KEYWORDS.stream().anyMatch(schema::has) || - schema.has("$ref") || - schema.has("type") + OBJECT_SCHEMA_KEYWORDS.stream().anyMatch(schema::has) || + ARRAY_SCHEMA_KEYWORDS.stream().anyMatch(schema::has) || + COMPOSED_SCHEMA_KEYWORDS.stream().anyMatch(schema::has) || + schema.has("$ref") || + schema.has("type") ); } @@ -121,12 +124,12 @@ private static JSONObject wrapSchemaInData(final JSONObject schema, final EventT properties.put("data_type", new JSONObject().put("type", "string")); properties.put("data_op", new JSONObject().put("type", "string") - .put("enum", Arrays.asList(new String[] { "C", "U", "D", "S" }))); + .put("enum", Arrays.asList(new String[]{"C", "U", "D", "S"}))); properties.put(DATA_CHANGE_WRAP_FIELD, schema); wrapper.put("additionalProperties", false); - addToRequired(wrapper, new String[]{ "data_type", "data_op", "data" }); + addToRequired(wrapper, new String[]{"data_type", "data_op", "data"}); return wrapper; } @@ -153,7 +156,7 @@ private static JSONObject addMetadata(final JSONObject schema, final EventType e .put("items", uuid); final JSONObject eventTypeString = new JSONObject() .put("type", "string") - .put("enum", Arrays.asList(new String[] { eventType.getName() })); + .put("enum", Arrays.asList(new String[]{eventType.getName()})); final JSONObject string = new JSONObject().put("type", "string"); final JSONObject dateTime = new JSONObject() .put("type", "string"); @@ -164,6 +167,7 @@ private static JSONObject addMetadata(final JSONObject schema, final EventType e metadataProperties.put("parent_eids", arrayOfUUIDs); metadataProperties.put("flow_id", string); metadataProperties.put("partition", string); + metadataProperties.put("partition_compaction_key", string); metadata.put("type", "object"); metadata.put("properties", metadataProperties); @@ -172,7 +176,7 @@ private static JSONObject addMetadata(final JSONObject schema, final EventType e schema.getJSONObject("properties").put("metadata", metadata); - addToRequired(schema, new String[]{ "metadata" }); + addToRequired(schema, new String[]{"metadata"}); return schema; } @@ -182,7 +186,7 @@ private static void addToRequired(final JSONObject schema, final String[] toBeRe final JSONArray currentRequired = schema.getJSONArray("required"); - for(int i = 0; i < currentRequired.length(); i++) { + for (int i = 0; i < currentRequired.length(); i++) { required.add(currentRequired.getString(i)); } diff --git a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java index 4e95514af4..9c3d8b0e75 100644 --- a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java +++ b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java @@ -234,70 +234,15 @@ public void whenPUTwithNullAudienceThen422() throws Exception { .andExpect(status().isUnprocessableEntity()); } - @Test - public void whenPostCompactedETWithoutCompactionKeysThen422() throws Exception { - final EventType eventType = EventTypeTestBuilder.builder() - .cleanupPolicy(CleanupPolicy.COMPACT) - .build(); - postEventType(eventType).andExpect(status().isUnprocessableEntity()); - } - - @Test - public void whenCompactionKeysSpecifiedForNoneCompactedETThen422() throws Exception { - final EventType eventType = EventTypeTestBuilder.builder() - .partitionCompactionKeys(ImmutableList.of("key")) - .schema("{\"type\":\"object\",\"properties\":{\"key\":{\"type\":\"string\"}}}") - .build(); - postEventType(eventType).andExpect(status().isUnprocessableEntity()); - } - - @Test - public void whenCompactionKeysNotSpecifiedInSchemaThen422() throws Exception { - final EventType eventType = EventTypeTestBuilder.builder() - .cleanupPolicy(CleanupPolicy.COMPACT) - .partitionCompactionKeys(ImmutableList.of("key")) - .build(); - postEventType(eventType).andExpect(status().isUnprocessableEntity()); - } - @Test public void whenPutEventTypeWithChangedCleanupPolicyThen422() throws Exception { final EventType originalEventType = EventTypeTestBuilder.builder() .cleanupPolicy(CleanupPolicy.COMPACT) - .partitionCompactionKeys(ImmutableList.of("key")) - .schema("{\"type\":\"object\",\"properties\":{\"key\":{\"type\":\"string\"}}}") .build(); final EventType updatedEventType = EventTypeTestBuilder.builder() .name(originalEventType.getName()) .cleanupPolicy(CleanupPolicy.DELETE) - .partitionCompactionKeys(ImmutableList.of("key")) - .schema("{\"type\":\"object\",\"properties\":{\"key\":{\"type\":\"string\"}}}") - .build(); - - doReturn(originalEventType).when(eventTypeRepository).findByName(any()); - - putEventType(updatedEventType, originalEventType.getName()) - .andExpect(status().isUnprocessableEntity()); - } - - @Test - public void whenPutEventTypeWithChangedCompactionKeysThen422() throws Exception { - final EventType originalEventType = EventTypeTestBuilder.builder() - .cleanupPolicy(CleanupPolicy.COMPACT) - .partitionCompactionKeys(ImmutableList.of("key")) - .schema("{\"type\":\"object\",\"properties\":{" + - "\"key\":{\"type\":\"string\"}," + - "\"key2\":{\"type\":\"string\"}}}") - .build(); - - final EventType updatedEventType = EventTypeTestBuilder.builder() - .name(originalEventType.getName()) - .cleanupPolicy(CleanupPolicy.COMPACT) - .partitionCompactionKeys(ImmutableList.of("key2")) - .schema("{\"type\":\"object\",\"properties\":{" + - "\"key\":{\"type\":\"string\"}," + - "\"key2\":{\"type\":\"string\"}}}") .build(); doReturn(originalEventType).when(eventTypeRepository).findByName(any()); diff --git a/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java b/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java index b3010bc1c4..716f921886 100644 --- a/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java +++ b/src/test/java/org/zalando/nakadi/utils/EventTypeTestBuilder.java @@ -4,18 +4,18 @@ import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.json.JSONObject; +import org.zalando.nakadi.domain.Audience; import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.CompatibilityMode; import org.zalando.nakadi.domain.EnrichmentStrategyDescriptor; -import org.zalando.nakadi.domain.Audience; import org.zalando.nakadi.domain.EventCategory; import org.zalando.nakadi.domain.EventType; -import org.zalando.nakadi.domain.ResourceAuthorization; import org.zalando.nakadi.domain.EventTypeBase; import org.zalando.nakadi.domain.EventTypeOptions; import org.zalando.nakadi.domain.EventTypeSchema; import org.zalando.nakadi.domain.EventTypeSchemaBase; import org.zalando.nakadi.domain.EventTypeStatistics; +import org.zalando.nakadi.domain.ResourceAuthorization; import org.zalando.nakadi.domain.ValidationStrategyConfiguration; import org.zalando.nakadi.partitioning.PartitionStrategy; @@ -39,7 +39,6 @@ public class EventTypeTestBuilder { private EventTypeOptions options; private CompatibilityMode compatibilityMode; private CleanupPolicy cleanupPolicy; - private List partitionCompactionKeys; private DateTime createdAt; private DateTime updatedAt; private Audience audience; @@ -137,11 +136,6 @@ public EventTypeTestBuilder cleanupPolicy(final CleanupPolicy cleanupPolicy) { return this; } - public EventTypeTestBuilder partitionCompactionKeys(final List partitionCompactionKeys) { - this.partitionCompactionKeys = partitionCompactionKeys; - return this; - } - public EventTypeTestBuilder createdAt(final DateTime createdAt) { this.createdAt = createdAt; return this; @@ -165,7 +159,7 @@ public EventTypeTestBuilder audience(final Audience audience) { public EventType build() { final EventTypeBase eventTypeBase = new EventTypeBase(name, owningApplication, category, validationStrategies, enrichmentStrategies, partitionStrategy, partitionKeyFields, schema, - defaultStatistic, options, compatibilityMode, cleanupPolicy, partitionCompactionKeys); + defaultStatistic, options, compatibilityMode, cleanupPolicy); eventTypeBase.setAuthorization(authorization); eventTypeBase.setAudience(audience); return new EventType(eventTypeBase, this.schema.getVersion().toString(), this.createdAt, this.updatedAt); From c11538b79b6b6a7aad4c60387e6930823480da11 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 13:48:16 +0200 Subject: [PATCH 12/26] ARUHA-1757: fixed test; --- .../java/org/zalando/nakadi/webservice/UserJourneyAT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/acceptance-test/java/org/zalando/nakadi/webservice/UserJourneyAT.java b/src/acceptance-test/java/org/zalando/nakadi/webservice/UserJourneyAT.java index 91968aef60..3e47b03ee6 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/webservice/UserJourneyAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/webservice/UserJourneyAT.java @@ -93,7 +93,6 @@ public void userJourneyM1() throws InterruptedException, IOException { .body("audience", equalTo("external-public")) .body("ordering_key_fields", equalTo(Lists.newArrayList("foo", "bar.baz"))) .body("cleanup_policy", equalTo("delete")) - .body("partition_compaction_keys", equalTo(ImmutableList.of())) .body("schema.type", equalTo("json_schema")) .body("schema.schema", equalTo("{\"type\": \"object\", \"properties\": {\"foo\": " + "{\"type\": \"string\"}, \"bar\": {\"type\": \"object\", \"properties\": " + From f1c5e4e59d43f258545b4c83f331da3458dd47b9 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 13:49:41 +0200 Subject: [PATCH 13/26] ARUHA-1757: reverted change; --- .../org/zalando/nakadi/partitioning/HashPartitionStrategy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/zalando/nakadi/partitioning/HashPartitionStrategy.java b/src/main/java/org/zalando/nakadi/partitioning/HashPartitionStrategy.java index 0fd0337a4f..3e6f782514 100644 --- a/src/main/java/org/zalando/nakadi/partitioning/HashPartitionStrategy.java +++ b/src/main/java/org/zalando/nakadi/partitioning/HashPartitionStrategy.java @@ -19,7 +19,7 @@ @Component public class HashPartitionStrategy implements PartitionStrategy { - public static final String DATA_PATH_PREFIX = JsonSchemaEnrichment.DATA_CHANGE_WRAP_FIELD + "."; + private static final String DATA_PATH_PREFIX = JsonSchemaEnrichment.DATA_CHANGE_WRAP_FIELD + "."; private final HashPartitionStrategyCrutch hashPartitioningCrutch; private final StringHash stringHash; From cb3b17ea169b088b644dcb66da7467424dc1f2e9 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 13:50:53 +0200 Subject: [PATCH 14/26] ARUHA-1757: added final; --- src/main/java/org/zalando/nakadi/service/EventPublisher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/zalando/nakadi/service/EventPublisher.java b/src/main/java/org/zalando/nakadi/service/EventPublisher.java index 372cd9285a..36c8bf1d8b 100644 --- a/src/main/java/org/zalando/nakadi/service/EventPublisher.java +++ b/src/main/java/org/zalando/nakadi/service/EventPublisher.java @@ -176,7 +176,7 @@ private void compact(final List batch, final EventType eventType) thr .getJSONObject("metadata") .getString("partition_compaction_key"); item.setEventKey(compactionKey); - } catch (JSONException e) { + } catch (final JSONException e) { item.updateStatusAndDetail(EventPublishingStatus.FAILED, "no compaction key found in metadata"); throw new CompactionException("No compaction key found in event metadata"); } From 7ab73d2309358778495a30321d864422a1bc4a68 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 15:08:06 +0200 Subject: [PATCH 15/26] ARUHA-1757: added compaction key presence validation to schema validation; --- .../org/zalando/nakadi/service/EventPublisher.java | 14 ++++---------- .../nakadi/validation/JsonSchemaEnrichment.java | 12 +++++++++++- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/src/main/java/org/zalando/nakadi/service/EventPublisher.java b/src/main/java/org/zalando/nakadi/service/EventPublisher.java index 36c8bf1d8b..cdd7e931ab 100644 --- a/src/main/java/org/zalando/nakadi/service/EventPublisher.java +++ b/src/main/java/org/zalando/nakadi/service/EventPublisher.java @@ -1,6 +1,5 @@ package org.zalando.nakadi.service; -import org.json.JSONException; import org.json.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -171,15 +170,10 @@ private void partition(final List batch, final EventType eventType) t private void compact(final List batch, final EventType eventType) throws CompactionException { if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { for (final BatchItem item : batch) { - try { - final String compactionKey = item.getEvent() - .getJSONObject("metadata") - .getString("partition_compaction_key"); - item.setEventKey(compactionKey); - } catch (final JSONException e) { - item.updateStatusAndDetail(EventPublishingStatus.FAILED, "no compaction key found in metadata"); - throw new CompactionException("No compaction key found in event metadata"); - } + final String compactionKey = item.getEvent() + .getJSONObject("metadata") + .getString("partition_compaction_key"); + item.setEventKey(compactionKey); } } } diff --git a/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java b/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java index bc05b4e2d0..f504ee5e25 100644 --- a/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java +++ b/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java @@ -4,15 +4,19 @@ import org.json.JSONArray; import org.json.JSONException; import org.json.JSONObject; +import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.CompatibilityMode; import org.zalando.nakadi.domain.EventType; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Set; +import static com.google.common.collect.Lists.newArrayList; + public class JsonSchemaEnrichment { public static final String DATA_CHANGE_WRAP_FIELD = "data"; @@ -169,9 +173,15 @@ private static JSONObject addMetadata(final JSONObject schema, final EventType e metadataProperties.put("partition", string); metadataProperties.put("partition_compaction_key", string); + final ArrayList requiredFields = newArrayList("eid", "occurred_at"); + if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { + metadataProperties.put("partition_compaction_key", string); + requiredFields.add("partition_compaction_key"); + } + metadata.put("type", "object"); metadata.put("properties", metadataProperties); - metadata.put("required", Arrays.asList(new String[]{"eid", "occurred_at"})); + metadata.put("required", requiredFields); metadata.put("additionalProperties", false); schema.getJSONObject("properties").put("metadata", metadata); From bdc16490a221289bb8881d2ed56ada58ab151e21 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 15:19:02 +0200 Subject: [PATCH 16/26] ARUHA-1757: added validation that compacted event type can't be undefined; --- .../org/zalando/nakadi/service/EventTypeService.java | 12 ++++++++++++ .../nakadi/controller/EventTypeControllerTest.java | 10 ++++++++++ 2 files changed, 22 insertions(+) diff --git a/src/main/java/org/zalando/nakadi/service/EventTypeService.java b/src/main/java/org/zalando/nakadi/service/EventTypeService.java index 29938a11a4..5f18ee7208 100644 --- a/src/main/java/org/zalando/nakadi/service/EventTypeService.java +++ b/src/main/java/org/zalando/nakadi/service/EventTypeService.java @@ -15,6 +15,7 @@ import org.springframework.transaction.TransactionException; import org.springframework.transaction.support.TransactionTemplate; import org.zalando.nakadi.config.NakadiSettings; +import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.CompatibilityMode; import org.zalando.nakadi.domain.EventCategory; import org.zalando.nakadi.domain.EventType; @@ -148,6 +149,7 @@ public void create(final EventTypeBase eventType) eventTypeOptionsValidator.checkRetentionTime(eventType.getOptions()); setDefaultEventTypeOptions(eventType); validateSchema(eventType); + validateCompaction(eventType); enrichment.validate(eventType); partitionResolver.validate(eventType); authorizationValidator.validateAuthorization(eventType.getAuthorization()); @@ -173,7 +175,17 @@ public void create(final EventTypeBase eventType) .put("compatibility_mode", eventType.getCompatibilityMode())); } + private void validateCompaction(final EventTypeBase eventType) throws + InvalidEventTypeException { + if (eventType.getCategory() == EventCategory.UNDEFINED && + eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { + throw new InvalidEventTypeException( + "cleanup_policy 'compact' is not available for 'undefined' event type category"); + } + } + private void validateCompactionUpdate(final EventType original, final EventTypeBase updatedET) { + validateCompaction(updatedET); if (original.getCleanupPolicy() != updatedET.getCleanupPolicy()) { throw new InvalidEventTypeException("cleanup_policy can not be changed"); } diff --git a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java index 9c3d8b0e75..30ca7af70d 100644 --- a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java +++ b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java @@ -17,6 +17,7 @@ import org.zalando.nakadi.domain.Audience; import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.EnrichmentStrategyDescriptor; +import org.zalando.nakadi.domain.EventCategory; import org.zalando.nakadi.domain.EventType; import org.zalando.nakadi.domain.EventTypeBase; import org.zalando.nakadi.domain.EventTypeOptions; @@ -234,6 +235,15 @@ public void whenPUTwithNullAudienceThen422() throws Exception { .andExpect(status().isUnprocessableEntity()); } + @Test + public void whenPostUndefinedEventTypeWithCompactCleanupPolicyThen422() throws Exception { + final EventType eventType = EventTypeTestBuilder.builder() + .cleanupPolicy(CleanupPolicy.COMPACT) + .category(EventCategory.UNDEFINED) + .build(); + postEventType(eventType).andExpect(status().isUnprocessableEntity()); + } + @Test public void whenPutEventTypeWithChangedCleanupPolicyThen422() throws Exception { final EventType originalEventType = EventTypeTestBuilder.builder() From 939733792a424c68b893a490349e03a6bf343b62 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 15:20:47 +0200 Subject: [PATCH 17/26] ARUHA-1757: fixed exception handler; --- .../zalando/nakadi/controller/TimelinesController.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/java/org/zalando/nakadi/controller/TimelinesController.java b/src/main/java/org/zalando/nakadi/controller/TimelinesController.java index ad38bcd153..0d1ab0c2df 100644 --- a/src/main/java/org/zalando/nakadi/controller/TimelinesController.java +++ b/src/main/java/org/zalando/nakadi/controller/TimelinesController.java @@ -62,15 +62,15 @@ public ResponseEntity getTimelines(@PathVariable("name") final String eventTy .collect(Collectors.toList())); } - @ExceptionHandler(UnableProcessException.class) + @ExceptionHandler({ + UnableProcessException.class, + TimelinesNotSupportedException.class}) public ResponseEntity unprocessable(final UnableProcessException ex, final NativeWebRequest request) { LOG.error(ex.getMessage(), ex); return Responses.create(MoreStatus.UNPROCESSABLE_ENTITY, ex.getMessage(), request); } - @ExceptionHandler({ - NotFoundException.class, - TimelinesNotSupportedException.class}) + @ExceptionHandler(NotFoundException.class) public ResponseEntity notFound(final MyNakadiRuntimeException1 ex, final NativeWebRequest request) { LOG.error(ex.getMessage(), ex); return Responses.create(Response.Status.NOT_FOUND, ex.getMessage(), request); From 21005ae9162c4b59f1b87b61617c75d710b19cb7 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 15:37:42 +0200 Subject: [PATCH 18/26] ARUHA-1757: revered back NotFoundException handling; --- .../org/zalando/nakadi/controller/TimelinesController.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/java/org/zalando/nakadi/controller/TimelinesController.java b/src/main/java/org/zalando/nakadi/controller/TimelinesController.java index 0d1ab0c2df..c58c3d329f 100644 --- a/src/main/java/org/zalando/nakadi/controller/TimelinesController.java +++ b/src/main/java/org/zalando/nakadi/controller/TimelinesController.java @@ -15,7 +15,6 @@ import org.zalando.nakadi.exceptions.runtime.AccessDeniedException; import org.zalando.nakadi.exceptions.runtime.ConflictException; import org.zalando.nakadi.exceptions.runtime.InconsistentStateException; -import org.zalando.nakadi.exceptions.runtime.MyNakadiRuntimeException1; import org.zalando.nakadi.exceptions.runtime.NotFoundException; import org.zalando.nakadi.exceptions.runtime.RepositoryProblemException; import org.zalando.nakadi.exceptions.runtime.TimelineException; @@ -71,7 +70,7 @@ public ResponseEntity unprocessable(final UnableProcessException ex, fi } @ExceptionHandler(NotFoundException.class) - public ResponseEntity notFound(final MyNakadiRuntimeException1 ex, final NativeWebRequest request) { + public ResponseEntity notFound(final NotFoundException ex, final NativeWebRequest request) { LOG.error(ex.getMessage(), ex); return Responses.create(Response.Status.NOT_FOUND, ex.getMessage(), request); } From 393c9ddccf1a3fad4b27191ff0a2a48606ac6651 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 16:06:43 +0200 Subject: [PATCH 19/26] ARUHA-1757: added min length to compaction key field; --- .../org/zalando/nakadi/validation/JsonSchemaEnrichment.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java b/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java index f504ee5e25..00669f12be 100644 --- a/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java +++ b/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java @@ -175,7 +175,10 @@ private static JSONObject addMetadata(final JSONObject schema, final EventType e final ArrayList requiredFields = newArrayList("eid", "occurred_at"); if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { - metadataProperties.put("partition_compaction_key", string); + final JSONObject compactionKey = new JSONObject() + .put("type", "string") + .put("minLength", 1); + metadataProperties.put("partition_compaction_key", compactionKey); requiredFields.add("partition_compaction_key"); } From 5363d3ff43bbfeea7dad99c638ff585d1d118e9f Mon Sep 17 00:00:00 2001 From: vstepanov Date: Mon, 9 Jul 2018 17:22:31 +0200 Subject: [PATCH 20/26] ARUHA-1757: fixed compaction key validation code; --- .../java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java b/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java index 00669f12be..33487c92d5 100644 --- a/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java +++ b/src/main/java/org/zalando/nakadi/validation/JsonSchemaEnrichment.java @@ -171,7 +171,6 @@ private static JSONObject addMetadata(final JSONObject schema, final EventType e metadataProperties.put("parent_eids", arrayOfUUIDs); metadataProperties.put("flow_id", string); metadataProperties.put("partition", string); - metadataProperties.put("partition_compaction_key", string); final ArrayList requiredFields = newArrayList("eid", "occurred_at"); if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { From 13337d86a4406e99e27428bb2719f6aff78d30c4 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Tue, 10 Jul 2018 17:43:10 +0200 Subject: [PATCH 21/26] ARUHA-1757: refactored code; added additional configuration to kafka topic; --- .../repository/kafka/KafkaRepositoryAT.java | 16 ++-- .../runtime/TopicConfigException.java | 4 + .../repository/KafkaRepositoryCreator.java | 10 +-- .../nakadi/repository/NakadiTopicConfig.java | 35 ++++++++ .../nakadi/repository/TopicRepository.java | 4 +- .../repository/kafka/KafkaTopicConfig.java | 83 ++++++++++++++++++ .../kafka/KafkaTopicConfigBuilder.java | 84 ++++++++++++++++++ .../kafka/KafkaTopicConfigFactory.java | 86 +++++++++++++++++++ .../kafka/KafkaTopicRepository.java | 48 ++++------- .../service/timeline/TimelineService.java | 18 ++-- src/main/resources/application.yml | 4 + .../controller/EventTypeControllerTest.java | 2 +- .../kafka/KafkaTopicRepositoryTest.java | 4 +- .../service/CursorOperationsServiceTest.java | 4 +- 14 files changed, 347 insertions(+), 55 deletions(-) create mode 100644 src/main/java/org/zalando/nakadi/repository/NakadiTopicConfig.java create mode 100644 src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfig.java create mode 100644 src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfigBuilder.java create mode 100644 src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfigFactory.java diff --git a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java index ef0a03feec..f456f70d0f 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java @@ -15,6 +15,7 @@ import org.zalando.nakadi.domain.BatchItem; import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.EventPublishingStatus; +import org.zalando.nakadi.repository.NakadiTopicConfig; import org.zalando.nakadi.repository.zookeeper.ZooKeeperHolder; import org.zalando.nakadi.repository.zookeeper.ZookeeperSettings; import org.zalando.nakadi.util.UUIDGenerator; @@ -24,6 +25,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import static org.echocat.jomon.runtime.concurrent.Retryer.executeWithRetry; @@ -68,6 +70,8 @@ public class KafkaRepositoryAT extends BaseAT { private ZookeeperSettings zookeeperSettings; private KafkaTestHelper kafkaHelper; private KafkaTopicRepository kafkaTopicRepository; + private NakadiTopicConfig defaultTopicConfig; + private KafkaTopicConfigFactory kafkaTopicConfigFactory; @Before public void setup() { @@ -91,14 +95,17 @@ public void setup() { zookeeperSettings = new ZookeeperSettings(ZK_SESSION_TIMEOUT, ZK_CONNECTION_TIMEOUT); kafkaHelper = new KafkaTestHelper(KAFKA_URL); kafkaTopicRepository = createKafkaTopicRepository(); + defaultTopicConfig = new NakadiTopicConfig(DEFAULT_PARTITION_COUNT, DEFAULT_CLEANUP_POLICY, + Optional.of(DEFAULT_RETENTION_TIME)); + kafkaTopicConfigFactory = new KafkaTopicConfigFactory(new UUIDGenerator(), DEFAULT_REPLICA_FACTOR, + DEFAULT_TOPIC_ROTATION, 1L, 1L, 1L); } @Test(timeout = 10000) @SuppressWarnings("unchecked") public void whenCreateTopicThenTopicIsCreated() throws Exception { // ACT // - final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, DEFAULT_RETENTION_TIME, - DEFAULT_CLEANUP_POLICY); + final String topicName = kafkaTopicRepository.createTopic(defaultTopicConfig); // ASSERT // executeWithRetry(() -> { @@ -168,8 +175,7 @@ public void whenBulkSendSuccessfullyThenUpdateBatchItemStatus() throws Exception @SuppressWarnings("unchecked") public void whenCreateTopicWithRetentionTime() throws Exception { // ACT // - final String topicName = kafkaTopicRepository.createTopic(DEFAULT_PARTITION_COUNT, DEFAULT_RETENTION_TIME, - DEFAULT_CLEANUP_POLICY); + final String topicName = kafkaTopicRepository.createTopic(defaultTopicConfig); // ASSERT // executeWithRetry(() -> Assert.assertEquals( @@ -210,7 +216,7 @@ private KafkaTopicRepository createKafkaTopicRepository() { nakadiSettings, kafkaSettings, zookeeperSettings, - new UUIDGenerator()); + kafkaTopicConfigFactory); } } diff --git a/src/main/java/org/zalando/nakadi/exceptions/runtime/TopicConfigException.java b/src/main/java/org/zalando/nakadi/exceptions/runtime/TopicConfigException.java index 0695538482..a0379ed1d8 100644 --- a/src/main/java/org/zalando/nakadi/exceptions/runtime/TopicConfigException.java +++ b/src/main/java/org/zalando/nakadi/exceptions/runtime/TopicConfigException.java @@ -6,4 +6,8 @@ public TopicConfigException(final String message, final Exception e) { super(message, e); } + public TopicConfigException(final String message) { + super(message); + } + } diff --git a/src/main/java/org/zalando/nakadi/repository/KafkaRepositoryCreator.java b/src/main/java/org/zalando/nakadi/repository/KafkaRepositoryCreator.java index 8576d15783..731bba8ef1 100644 --- a/src/main/java/org/zalando/nakadi/repository/KafkaRepositoryCreator.java +++ b/src/main/java/org/zalando/nakadi/repository/KafkaRepositoryCreator.java @@ -12,10 +12,10 @@ import org.zalando.nakadi.repository.kafka.KafkaFactory; import org.zalando.nakadi.repository.kafka.KafkaLocationManager; import org.zalando.nakadi.repository.kafka.KafkaSettings; +import org.zalando.nakadi.repository.kafka.KafkaTopicConfigFactory; import org.zalando.nakadi.repository.kafka.KafkaTopicRepository; import org.zalando.nakadi.repository.zookeeper.ZooKeeperHolder; import org.zalando.nakadi.repository.zookeeper.ZookeeperSettings; -import org.zalando.nakadi.util.UUIDGenerator; import java.util.Comparator; import java.util.List; @@ -27,19 +27,19 @@ public class KafkaRepositoryCreator implements TopicRepositoryCreator { private final NakadiSettings nakadiSettings; private final KafkaSettings kafkaSettings; private final ZookeeperSettings zookeeperSettings; - private final UUIDGenerator uuidGenerator; + private final KafkaTopicConfigFactory kafkaTopicConfigFactory; private final MetricRegistry metricRegistry; @Autowired public KafkaRepositoryCreator(final NakadiSettings nakadiSettings, final KafkaSettings kafkaSettings, final ZookeeperSettings zookeeperSettings, - final UUIDGenerator uuidGenerator, + final KafkaTopicConfigFactory kafkaTopicConfigFactory, final MetricRegistry metricRegistry) { this.nakadiSettings = nakadiSettings; this.kafkaSettings = kafkaSettings; this.zookeeperSettings = zookeeperSettings; - this.uuidGenerator = uuidGenerator; + this.kafkaTopicConfigFactory = kafkaTopicConfigFactory; this.metricRegistry = metricRegistry; } @@ -55,7 +55,7 @@ public TopicRepository createTopicRepository(final Storage storage) throws Topic final KafkaFactory kafkaFactory = new KafkaFactory(new KafkaLocationManager(zooKeeperHolder, kafkaSettings), metricRegistry); final KafkaTopicRepository kafkaTopicRepository = new KafkaTopicRepository(zooKeeperHolder, - kafkaFactory, nakadiSettings, kafkaSettings, zookeeperSettings, uuidGenerator); + kafkaFactory, nakadiSettings, kafkaSettings, zookeeperSettings, kafkaTopicConfigFactory); // check that it does work kafkaTopicRepository.listTopics(); return kafkaTopicRepository; diff --git a/src/main/java/org/zalando/nakadi/repository/NakadiTopicConfig.java b/src/main/java/org/zalando/nakadi/repository/NakadiTopicConfig.java new file mode 100644 index 0000000000..c930d6e17c --- /dev/null +++ b/src/main/java/org/zalando/nakadi/repository/NakadiTopicConfig.java @@ -0,0 +1,35 @@ +package org.zalando.nakadi.repository; + +import org.zalando.nakadi.domain.CleanupPolicy; + +import java.util.Optional; + +public class NakadiTopicConfig { + + private final int partitionCount; + + private final CleanupPolicy cleanupPolicy; + + private final Optional retentionTimeMs; + + public NakadiTopicConfig(final int partitionCount, + final CleanupPolicy cleanupPolicy, + final Optional retentionTimeMs) { + this.partitionCount = partitionCount; + this.cleanupPolicy = cleanupPolicy; + this.retentionTimeMs = retentionTimeMs; + } + + public int getPartitionCount() { + return partitionCount; + } + + public CleanupPolicy getCleanupPolicy() { + return cleanupPolicy; + } + + public Optional getRetentionTimeMs() { + return retentionTimeMs; + } + +} diff --git a/src/main/java/org/zalando/nakadi/repository/TopicRepository.java b/src/main/java/org/zalando/nakadi/repository/TopicRepository.java index 5cd86cced9..cb0c55a78a 100644 --- a/src/main/java/org/zalando/nakadi/repository/TopicRepository.java +++ b/src/main/java/org/zalando/nakadi/repository/TopicRepository.java @@ -1,7 +1,6 @@ package org.zalando.nakadi.repository; import org.zalando.nakadi.domain.BatchItem; -import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.NakadiCursor; import org.zalando.nakadi.domain.PartitionEndStatistics; import org.zalando.nakadi.domain.PartitionStatistics; @@ -39,8 +38,7 @@ public String getPartition() { } } - String createTopic(int partitionCount, Long retentionTimeMs, CleanupPolicy cleanupPolicy) - throws TopicCreationException; + String createTopic(NakadiTopicConfig nakadiTopicConfig) throws TopicCreationException, TopicConfigException; void deleteTopic(String topic) throws TopicDeletionException; diff --git a/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfig.java b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfig.java new file mode 100644 index 0000000000..c91833f001 --- /dev/null +++ b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfig.java @@ -0,0 +1,83 @@ +package org.zalando.nakadi.repository.kafka; + +import kafka.admin.RackAwareMode; + +import java.util.Optional; + +public class KafkaTopicConfig { + + private final String topicName; + + private final int partitionCount; + + private final int replicaFactor; + + private final String cleanupPolicy; + + private final long segmentMs; + + private final Optional retentionMs; + + private final Optional segmentBytes; + + private final Optional minCompactionLagMs; + + private final RackAwareMode rackAwareMode; + + public KafkaTopicConfig(final String topicName, + final int partitionCount, + final int replicaFactor, + final String cleanupPolicy, + final long segmentMs, + final Optional retentionMs, + final Optional segmentBytes, + final Optional minCompactionLagMs, + final RackAwareMode rackAwareMode) { + this.topicName = topicName; + this.partitionCount = partitionCount; + this.replicaFactor = replicaFactor; + this.cleanupPolicy = cleanupPolicy; + this.segmentMs = segmentMs; + this.retentionMs = retentionMs; + this.segmentBytes = segmentBytes; + this.minCompactionLagMs = minCompactionLagMs; + this.rackAwareMode = rackAwareMode; + } + + public String getTopicName() { + return topicName; + } + + public int getPartitionCount() { + return partitionCount; + } + + public int getReplicaFactor() { + return replicaFactor; + } + + public String getCleanupPolicy() { + return cleanupPolicy; + } + + public long getSegmentMs() { + return segmentMs; + } + + public Optional getRetentionMs() { + return retentionMs; + } + + public Optional getSegmentBytes() { + return segmentBytes; + } + + public Optional getMinCompactionLagMs() { + return minCompactionLagMs; + } + + public RackAwareMode getRackAwareMode() { + return rackAwareMode; + } + +} diff --git a/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfigBuilder.java b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfigBuilder.java new file mode 100644 index 0000000000..b4cd8fe131 --- /dev/null +++ b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfigBuilder.java @@ -0,0 +1,84 @@ +package org.zalando.nakadi.repository.kafka; + +import kafka.admin.RackAwareMode; + +import java.util.Optional; + +public final class KafkaTopicConfigBuilder { + + private String topicName; + private int partitionCount; + private int replicaFactor; + private String cleanupPolicy; + private long segmentMs; + private Long retentionMs; + private Long segmentBytes; + private Long minCompactionLagMs; + private RackAwareMode rackAwareMode; + + private KafkaTopicConfigBuilder() { + } + + public static KafkaTopicConfigBuilder builder() { + return new KafkaTopicConfigBuilder(); + } + + public KafkaTopicConfigBuilder withTopicName(final String topicName) { + this.topicName = topicName; + return this; + } + + public KafkaTopicConfigBuilder withPartitionCount(final int partitionCount) { + this.partitionCount = partitionCount; + return this; + } + + public KafkaTopicConfigBuilder withReplicaFactor(final int replicaFactor) { + this.replicaFactor = replicaFactor; + return this; + } + + public KafkaTopicConfigBuilder withCleanupPolicy(final String cleanupPolicy) { + this.cleanupPolicy = cleanupPolicy; + return this; + } + + public KafkaTopicConfigBuilder withSegmentMs(final long segmentMs) { + this.segmentMs = segmentMs; + return this; + } + + public KafkaTopicConfigBuilder withRetentionMs(final Long retentionMs) { + this.retentionMs = retentionMs; + return this; + } + + public KafkaTopicConfigBuilder withSegmentBytes(final Long segmentBytes) { + this.segmentBytes = segmentBytes; + return this; + } + + public KafkaTopicConfigBuilder withMinCompactionLagMs(final Long minCompactionLagMs) { + this.minCompactionLagMs = minCompactionLagMs; + return this; + } + + public KafkaTopicConfigBuilder withRackAwareMode(final RackAwareMode rackAwareMode) { + this.rackAwareMode = rackAwareMode; + return this; + } + + public KafkaTopicConfig build() { + return new KafkaTopicConfig( + topicName, + partitionCount, + replicaFactor, + cleanupPolicy, + segmentMs, + Optional.ofNullable(retentionMs), + Optional.ofNullable(segmentBytes), + Optional.ofNullable(minCompactionLagMs), + rackAwareMode); + } + +} diff --git a/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfigFactory.java b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfigFactory.java new file mode 100644 index 0000000000..bfb815b60f --- /dev/null +++ b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicConfigFactory.java @@ -0,0 +1,86 @@ +package org.zalando.nakadi.repository.kafka; + +import kafka.admin.RackAwareMode; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Component; +import org.zalando.nakadi.domain.CleanupPolicy; +import org.zalando.nakadi.exceptions.runtime.TopicConfigException; +import org.zalando.nakadi.repository.NakadiTopicConfig; +import org.zalando.nakadi.util.UUIDGenerator; + +import java.util.Properties; + +@Component +public class KafkaTopicConfigFactory { + + private final UUIDGenerator uuidGenerator; + + private final int defaultTopicReplicaFactor; + private final long defaultTopicRotationMs; + private final long compactedTopicRotationMs; + private final long compactedTopicSegmentBytes; + private final long compactedTopicCompactionLagMs; + + @Autowired + public KafkaTopicConfigFactory( + final UUIDGenerator uuidGenerator, + @Value("${nakadi.topic.default.replicaFactor}") final int defaultTopicReplicaFactor, + @Value("${nakadi.topic.default.rotationMs}") final long defaultTopicRotationMs, + @Value("${nakadi.topic.compacted.rotationMs}") final long compactedTopicRotationMs, + @Value("${nakadi.topic.compacted.segmentBytes}") final long compactedTopicSegmentBytes, + @Value("${nakadi.topic.compacted.compactionLagMs}") final long compactedTopicCompactionLagMs) { + this.uuidGenerator = uuidGenerator; + this.defaultTopicReplicaFactor = defaultTopicReplicaFactor; + this.defaultTopicRotationMs = defaultTopicRotationMs; + this.compactedTopicRotationMs = compactedTopicRotationMs; + this.compactedTopicSegmentBytes = compactedTopicSegmentBytes; + this.compactedTopicCompactionLagMs = compactedTopicCompactionLagMs; + } + + public KafkaTopicConfig createKafkaTopicConfig(final NakadiTopicConfig topicConfig) throws TopicConfigException { + + // set common values + final KafkaTopicConfigBuilder configBuilder = KafkaTopicConfigBuilder.builder() + .withTopicName(uuidGenerator.randomUUID().toString()) + .withPartitionCount(topicConfig.getPartitionCount()) + .withReplicaFactor(defaultTopicReplicaFactor) + .withRackAwareMode(RackAwareMode.Safe$.MODULE$); + + if (topicConfig.getCleanupPolicy() == CleanupPolicy.COMPACT) { + // set values specific for cleanup policy 'compact' + configBuilder + .withCleanupPolicy("compact") + .withSegmentMs(compactedTopicRotationMs) + .withSegmentBytes(compactedTopicSegmentBytes) + .withMinCompactionLagMs(compactedTopicCompactionLagMs); + + } else if (topicConfig.getCleanupPolicy() == CleanupPolicy.DELETE) { + // set values specific for cleanup policy 'delete' + configBuilder + .withCleanupPolicy("delete") + .withRetentionMs(topicConfig.getRetentionTimeMs() + .orElseThrow(() -> new TopicConfigException("retention time should be specified " + + "for topic with cleanup policy 'delete'"))) + .withSegmentMs(defaultTopicRotationMs); + } + return configBuilder.build(); + } + + public Properties createKafkaTopicLevelProperties(final KafkaTopicConfig kafkaTopicConfig) { + final Properties topicConfig = new Properties(); + + topicConfig.setProperty("segment.ms", Long.toString(kafkaTopicConfig.getSegmentMs())); + topicConfig.setProperty("cleanup.policy", kafkaTopicConfig.getCleanupPolicy()); + + kafkaTopicConfig.getRetentionMs() + .ifPresent(v -> topicConfig.setProperty("retention.ms", Long.toString(v))); + kafkaTopicConfig.getSegmentBytes() + .ifPresent(v -> topicConfig.setProperty("segment.bytes", Long.toString(v))); + kafkaTopicConfig.getMinCompactionLagMs() + .ifPresent(v -> topicConfig.setProperty("min.compaction.lag.ms", Long.toString(v))); + + return topicConfig; + } + +} diff --git a/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java index e59253d5ad..90023698f9 100644 --- a/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java +++ b/src/main/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepository.java @@ -2,7 +2,6 @@ import com.google.common.base.Preconditions; import kafka.admin.AdminUtils; -import kafka.admin.RackAwareMode; import kafka.server.ConfigType; import kafka.utils.ZkUtils; import org.apache.kafka.clients.consumer.Consumer; @@ -22,7 +21,6 @@ import org.slf4j.LoggerFactory; import org.zalando.nakadi.config.NakadiSettings; import org.zalando.nakadi.domain.BatchItem; -import org.zalando.nakadi.domain.CleanupPolicy; import org.zalando.nakadi.domain.EventPublishingStatus; import org.zalando.nakadi.domain.EventPublishingStep; import org.zalando.nakadi.domain.NakadiCursor; @@ -37,10 +35,10 @@ import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; import org.zalando.nakadi.repository.EventConsumer; +import org.zalando.nakadi.repository.NakadiTopicConfig; import org.zalando.nakadi.repository.TopicRepository; import org.zalando.nakadi.repository.zookeeper.ZooKeeperHolder; import org.zalando.nakadi.repository.zookeeper.ZookeeperSettings; -import org.zalando.nakadi.util.UUIDGenerator; import javax.annotation.Nullable; import java.util.ArrayList; @@ -81,20 +79,20 @@ public class KafkaTopicRepository implements TopicRepository { private final KafkaSettings kafkaSettings; private final ZookeeperSettings zookeeperSettings; private final ConcurrentMap circuitBreakers; - private final UUIDGenerator uuidGenerator; + private final KafkaTopicConfigFactory kafkaTopicConfigFactory; public KafkaTopicRepository(final ZooKeeperHolder zkFactory, final KafkaFactory kafkaFactory, final NakadiSettings nakadiSettings, final KafkaSettings kafkaSettings, final ZookeeperSettings zookeeperSettings, - final UUIDGenerator uuidGenerator) { + final KafkaTopicConfigFactory kafkaTopicConfigFactory) { this.zkFactory = zkFactory; this.kafkaFactory = kafkaFactory; this.nakadiSettings = nakadiSettings; this.kafkaSettings = kafkaSettings; this.zookeeperSettings = zookeeperSettings; - this.uuidGenerator = uuidGenerator; + this.kafkaTopicConfigFactory = kafkaTopicConfigFactory; this.circuitBreakers = new ConcurrentHashMap<>(); } @@ -109,38 +107,23 @@ public List listTopics() throws TopicRepositoryException { } @Override - public String createTopic(final int partitionCount, final Long retentionTimeMs, final CleanupPolicy cleanupPolicy) - throws TopicCreationException { - if (retentionTimeMs == null) { - throw new IllegalArgumentException("Retention time can not be null"); - } - final String topicName = uuidGenerator.randomUUID().toString(); - createTopic(topicName, - partitionCount, - nakadiSettings.getDefaultTopicReplicaFactor(), - retentionTimeMs, - nakadiSettings.getDefaultTopicRotationMs(), - cleanupPolicy == CleanupPolicy.COMPACT ? "compact" : "delete"); - return topicName; - } + public String createTopic(final NakadiTopicConfig nakadiTopicConfig) throws TopicCreationException { - private void createTopic(final String topic, final int partitionsNum, final int replicaFactor, - final long retentionMs, final long rotationMs, final String cleanupPolicy) - throws TopicCreationException { + final KafkaTopicConfig kafkaTopicConfig = kafkaTopicConfigFactory.createKafkaTopicConfig(nakadiTopicConfig); try { doWithZkUtils(zkUtils -> { - final Properties topicConfig = new Properties(); - topicConfig.setProperty("retention.ms", Long.toString(retentionMs)); - topicConfig.setProperty("segment.ms", Long.toString(rotationMs)); - topicConfig.setProperty("cleanup.policy", cleanupPolicy); - AdminUtils.createTopic(zkUtils, topic, partitionsNum, replicaFactor, topicConfig, - RackAwareMode.Safe$.MODULE$); + AdminUtils.createTopic(zkUtils, + kafkaTopicConfig.getTopicName(), + kafkaTopicConfig.getPartitionCount(), + kafkaTopicConfig.getReplicaFactor(), + kafkaTopicConfigFactory.createKafkaTopicLevelProperties(kafkaTopicConfig), + kafkaTopicConfig.getRackAwareMode()); }); } catch (final TopicExistsException e) { - throw new TopicCreationException("Topic with name " + topic + + throw new TopicCreationException("Topic with name " + kafkaTopicConfig.getTopicName() + " already exists (or wasn't completely removed yet)", e); } catch (final Exception e) { - throw new TopicCreationException("Unable to create topic " + topic, e); + throw new TopicCreationException("Unable to create topic " + kafkaTopicConfig.getTopicName(), e); } // Next step is to wait for topic initialization. On can not skip this task, cause kafka instances may not // receive information about topic creation, which in turn will block publishing. @@ -148,7 +131,7 @@ private void createTopic(final String topic, final int partitionsNum, final int final long timeoutMillis = TimeUnit.SECONDS.toMillis(5); final Boolean allowsConsumption = Retryer.executeWithRetry(() -> { try (Consumer consumer = kafkaFactory.getConsumer()) { - return null != consumer.partitionsFor(topic); + return null != consumer.partitionsFor(kafkaTopicConfig.getTopicName()); } }, new RetryForSpecifiedTimeStrategy(timeoutMillis) @@ -158,6 +141,7 @@ private void createTopic(final String topic, final int partitionsNum, final int throw new TopicCreationException("Failed to confirm topic creation within " + timeoutMillis + " millis"); } + return kafkaTopicConfig.getTopicName(); } @Override diff --git a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java index fe078e28aa..e3cb0ba2d5 100644 --- a/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java +++ b/src/main/java/org/zalando/nakadi/service/timeline/TimelineService.java @@ -34,6 +34,7 @@ import org.zalando.nakadi.exceptions.runtime.ServiceTemporarilyUnavailableException; import org.zalando.nakadi.exceptions.runtime.TimelineException; import org.zalando.nakadi.exceptions.runtime.TimelinesNotSupportedException; +import org.zalando.nakadi.exceptions.runtime.TopicConfigException; import org.zalando.nakadi.exceptions.runtime.TopicCreationException; import org.zalando.nakadi.exceptions.runtime.TopicDeletionException; import org.zalando.nakadi.exceptions.runtime.TopicRepositoryException; @@ -42,6 +43,7 @@ import org.zalando.nakadi.plugin.api.authz.Resource; import org.zalando.nakadi.repository.EventConsumer; import org.zalando.nakadi.repository.MultiTimelineEventConsumer; +import org.zalando.nakadi.repository.NakadiTopicConfig; import org.zalando.nakadi.repository.TopicRepository; import org.zalando.nakadi.repository.TopicRepositoryHolder; import org.zalando.nakadi.repository.db.EventTypeCache; @@ -56,6 +58,7 @@ import java.util.Date; import java.util.List; import java.util.ListIterator; +import java.util.Optional; import java.util.stream.Collectors; @Service @@ -128,13 +131,15 @@ public void createTimeline(final String eventTypeName, final String storageId) final List partitionStatistics = currentTopicRepo.loadTopicStatistics(Collections.singleton(activeTimeline)); - final String newTopic = nextTopicRepo.createTopic(partitionStatistics.size(), - eventType.getOptions().getRetentionTime(), eventType.getCleanupPolicy()); + final NakadiTopicConfig nakadiTopicConfig = new NakadiTopicConfig(partitionStatistics.size(), + eventType.getCleanupPolicy(), Optional.ofNullable(eventType.getOptions().getRetentionTime())); + final String newTopic = nextTopicRepo.createTopic(nakadiTopicConfig); final Timeline nextTimeline = Timeline.createTimeline(activeTimeline.getEventType(), activeTimeline.getOrder() + 1, storage, newTopic, new Date()); switchTimelines(activeTimeline, nextTimeline); - } catch (final TopicCreationException | ServiceTemporarilyUnavailableException | InternalNakadiException e) { + } catch (final TopicCreationException | TopicConfigException | ServiceTemporarilyUnavailableException | + InternalNakadiException e) { throw new TimelineException("Internal service error", e); } catch (final NoSuchEventTypeException e) { throw new NotFoundException("EventType \"" + eventTypeName + "\" does not exist"); @@ -154,14 +159,17 @@ public Timeline createDefaultTimeline(final EventTypeBase eventType, final int p } Storage storage = defaultStorage.getStorage(); + Optional retentionTime = Optional.ofNullable(eventType.getOptions().getRetentionTime()); if (eventType.getCleanupPolicy() == CleanupPolicy.COMPACT) { storage = storageDbRepository.getStorage(compactedStorageName).orElseThrow(() -> new TopicCreationException("No storage defined for compacted topics")); + retentionTime = Optional.empty(); } + final NakadiTopicConfig nakadiTopicConfig = new NakadiTopicConfig(partitionsCount, eventType.getCleanupPolicy(), + retentionTime); final TopicRepository repository = topicRepositoryHolder.getTopicRepository(storage); - final String topic = repository.createTopic(partitionsCount, eventType.getOptions().getRetentionTime(), - eventType.getCleanupPolicy()); + final String topic = repository.createTopic(nakadiTopicConfig); try { final Timeline timeline = Timeline.createTimeline(eventType.getName(), 1, storage, topic, new Date()); diff --git a/src/main/resources/application.yml b/src/main/resources/application.yml index f0a386caca..7f0da14bc5 100644 --- a/src/main/resources/application.yml +++ b/src/main/resources/application.yml @@ -48,6 +48,10 @@ nakadi: replicaFactor: 1 retentionMs: 172800000 # 2 days rotationMs: 86400000 # 1 day + compacted: + rotationMs: 3600000 # 1 hour + segmentBytes: 104857600 # 100 MB + compactionLagMs: 0 stream: timeoutMs: 31536000000 # 1 year :-P default.commitTimeout: 60 # 1 minute diff --git a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java index 30ca7af70d..7888ec792f 100644 --- a/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java +++ b/src/test/java/org/zalando/nakadi/controller/EventTypeControllerTest.java @@ -448,7 +448,7 @@ public void whenPostWithValidAuthorizationThenCreated() throws Exception { ImmutableList.of(new ResourceAuthorizationAttribute("type3", "value3")))); doReturn(eventType).when(eventTypeRepository).saveEventType(any(EventType.class)); - when(topicRepository.createTopic(anyInt(), any(), any())).thenReturn(randomUUID.toString()); + when(topicRepository.createTopic(any())).thenReturn(randomUUID.toString()); postEventType(eventType).andExpect(status().isCreated()); } diff --git a/src/test/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepositoryTest.java b/src/test/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepositoryTest.java index 6b697ea66b..205ce97def 100644 --- a/src/test/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepositoryTest.java +++ b/src/test/java/org/zalando/nakadi/repository/kafka/KafkaTopicRepositoryTest.java @@ -27,7 +27,6 @@ import org.zalando.nakadi.exceptions.runtime.EventPublishingException; import org.zalando.nakadi.repository.zookeeper.ZooKeeperHolder; import org.zalando.nakadi.repository.zookeeper.ZookeeperSettings; -import org.zalando.nakadi.util.UUIDGenerator; import org.zalando.nakadi.view.Cursor; import java.util.ArrayList; @@ -62,6 +61,7 @@ public class KafkaTopicRepositoryTest { private final NakadiSettings nakadiSettings = mock(NakadiSettings.class); private final KafkaSettings kafkaSettings = mock(KafkaSettings.class); private final ZookeeperSettings zookeeperSettings = mock(ZookeeperSettings.class); + private final KafkaTopicConfigFactory kafkaTopicConfigFactory = mock(KafkaTopicConfigFactory.class); private static final String KAFKA_CLIENT_ID = "application_name-topic_name"; @SuppressWarnings("unchecked") @@ -358,7 +358,7 @@ private KafkaTopicRepository createKafkaRepository(final KafkaFactory kafkaFacto nakadiSettings, kafkaSettings, zookeeperSettings, - new UUIDGenerator()); + kafkaTopicConfigFactory); } catch (final Exception e) { throw new RuntimeException(e); } diff --git a/src/test/java/org/zalando/nakadi/service/CursorOperationsServiceTest.java b/src/test/java/org/zalando/nakadi/service/CursorOperationsServiceTest.java index 071b8e51f5..66ca27ee74 100644 --- a/src/test/java/org/zalando/nakadi/service/CursorOperationsServiceTest.java +++ b/src/test/java/org/zalando/nakadi/service/CursorOperationsServiceTest.java @@ -11,11 +11,11 @@ import org.zalando.nakadi.repository.TopicRepository; import org.zalando.nakadi.repository.kafka.KafkaFactory; import org.zalando.nakadi.repository.kafka.KafkaSettings; +import org.zalando.nakadi.repository.kafka.KafkaTopicConfigFactory; import org.zalando.nakadi.repository.kafka.KafkaTopicRepository; import org.zalando.nakadi.repository.zookeeper.ZooKeeperHolder; import org.zalando.nakadi.repository.zookeeper.ZookeeperSettings; import org.zalando.nakadi.service.timeline.TimelineService; -import org.zalando.nakadi.util.UUIDGenerator; import javax.annotation.Nullable; import java.util.Arrays; @@ -325,7 +325,7 @@ private Timeline mockTimeline(final int order, @Nullable final Long latestOffset mock(NakadiSettings.class), mock(KafkaSettings.class), mock(ZookeeperSettings.class), - mock(UUIDGenerator.class)); + mock(KafkaTopicConfigFactory.class)); when(timelineService.getTopicRepository(timeline)).thenReturn(repository); return timeline; } From 7f908ae7f5e6591fa7909966736bafdaf1ae99bb Mon Sep 17 00:00:00 2001 From: vstepanov Date: Tue, 10 Jul 2018 18:15:30 +0200 Subject: [PATCH 22/26] ARUHA-1757: added test testing new properties; --- .../repository/kafka/KafkaRepositoryAT.java | 72 +++++++++++++------ .../repository/kafka/KafkaTestHelper.java | 2 +- 2 files changed, 53 insertions(+), 21 deletions(-) diff --git a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java index f456f70d0f..c6973161a0 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java +++ b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaRepositoryAT.java @@ -6,7 +6,6 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.echocat.jomon.runtime.concurrent.RetryForSpecifiedTimeStrategy; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; @@ -45,6 +44,9 @@ public class KafkaRepositoryAT extends BaseAT { private static final int DEFAULT_REPLICA_FACTOR = 1; private static final int MAX_TOPIC_PARTITION_COUNT = 10; private static final int DEFAULT_TOPIC_ROTATION = 50000000; + private static final int COMPACTED_TOPIC_ROTATION = 60000; + private static final int COMPACTED_TOPIC_SEGMENT_BYTES = 1000000; + private static final int COMPACTED_TOPIC_COMPACTION_LAG = 1000; private static final int DEFAULT_COMMIT_TIMEOUT = 60; private static final int ZK_SESSION_TIMEOUT = 30000; private static final int ZK_CONNECTION_TIMEOUT = 10000; @@ -94,16 +96,17 @@ public void setup() { KAFKA_LINGER_MS, KAFKA_ENABLE_AUTO_COMMIT, KAFKA_MAX_REQUEST_SIZE); zookeeperSettings = new ZookeeperSettings(ZK_SESSION_TIMEOUT, ZK_CONNECTION_TIMEOUT); kafkaHelper = new KafkaTestHelper(KAFKA_URL); - kafkaTopicRepository = createKafkaTopicRepository(); defaultTopicConfig = new NakadiTopicConfig(DEFAULT_PARTITION_COUNT, DEFAULT_CLEANUP_POLICY, Optional.of(DEFAULT_RETENTION_TIME)); kafkaTopicConfigFactory = new KafkaTopicConfigFactory(new UUIDGenerator(), DEFAULT_REPLICA_FACTOR, - DEFAULT_TOPIC_ROTATION, 1L, 1L, 1L); + DEFAULT_TOPIC_ROTATION, COMPACTED_TOPIC_ROTATION, COMPACTED_TOPIC_SEGMENT_BYTES, + COMPACTED_TOPIC_COMPACTION_LAG); + kafkaTopicRepository = createKafkaTopicRepository(); } @Test(timeout = 10000) @SuppressWarnings("unchecked") - public void whenCreateTopicThenTopicIsCreated() throws Exception { + public void whenCreateTopicThenTopicIsCreated() { // ACT // final String topicName = kafkaTopicRepository.createTopic(defaultTopicConfig); @@ -118,8 +121,51 @@ public void whenCreateTopicThenTopicIsCreated() throws Exception { partitionInfos.forEach(pInfo -> assertThat(pInfo.replicas(), arrayWithSize(DEFAULT_REPLICA_FACTOR))); + final Long retentionTime = KafkaTestHelper.getTopicRetentionTime(topicName, ZOOKEEPER_URL); + assertThat(retentionTime, equalTo(DEFAULT_RETENTION_TIME)); + final String cleanupPolicy = KafkaTestHelper.getTopicCleanupPolicy(topicName, ZOOKEEPER_URL); assertThat(cleanupPolicy, equalTo("delete")); + + final String segmentMs = KafkaTestHelper.getTopicProperty(topicName, ZOOKEEPER_URL, "segment.ms"); + assertThat(segmentMs, equalTo(String.valueOf(DEFAULT_TOPIC_ROTATION))); + }, + new RetryForSpecifiedTimeStrategy(5000).withExceptionsThatForceRetry(AssertionError.class) + .withWaitBetweenEachTry(500)); + } + + @Test(timeout = 10000) + @SuppressWarnings("unchecked") + public void whenCreateCompactedTopicThenTopicIsCreated() { + // ACT // + final NakadiTopicConfig compactedTopicConfig = new NakadiTopicConfig(DEFAULT_PARTITION_COUNT, + CleanupPolicy.COMPACT, Optional.empty()); + final String topicName = kafkaTopicRepository.createTopic(compactedTopicConfig); + + // ASSERT // + executeWithRetry(() -> { + final Map> topics = getAllTopics(); + assertThat(topics.keySet(), hasItem(topicName)); + + final List partitionInfos = topics.get(topicName); + assertThat(partitionInfos, hasSize(DEFAULT_PARTITION_COUNT)); + + partitionInfos.forEach(pInfo -> + assertThat(pInfo.replicas(), arrayWithSize(DEFAULT_REPLICA_FACTOR))); + + final String cleanupPolicy = KafkaTestHelper.getTopicCleanupPolicy(topicName, ZOOKEEPER_URL); + assertThat(cleanupPolicy, equalTo("compact")); + + final String segmentMs = KafkaTestHelper.getTopicProperty(topicName, ZOOKEEPER_URL, "segment.ms"); + assertThat(segmentMs, equalTo(String.valueOf(COMPACTED_TOPIC_ROTATION))); + + final String segmentBytes = KafkaTestHelper.getTopicProperty(topicName, ZOOKEEPER_URL, + "segment.bytes"); + assertThat(segmentBytes, equalTo(String.valueOf(COMPACTED_TOPIC_SEGMENT_BYTES))); + + final String compactionLag = KafkaTestHelper.getTopicProperty(topicName, ZOOKEEPER_URL, + "min.compaction.lag.ms"); + assertThat(compactionLag, equalTo(String.valueOf(COMPACTED_TOPIC_COMPACTION_LAG))); }, new RetryForSpecifiedTimeStrategy(5000).withExceptionsThatForceRetry(AssertionError.class) .withWaitBetweenEachTry(500)); @@ -127,7 +173,7 @@ public void whenCreateTopicThenTopicIsCreated() throws Exception { @Test(timeout = 20000) @SuppressWarnings("unchecked") - public void whenDeleteTopicThenTopicIsDeleted() throws Exception { + public void whenDeleteTopicThenTopicIsDeleted() { // ARRANGE // final String topicName = UUID.randomUUID().toString(); @@ -153,7 +199,7 @@ public void whenDeleteTopicThenTopicIsDeleted() throws Exception { } @Test(timeout = 10000) - public void whenBulkSendSuccessfullyThenUpdateBatchItemStatus() throws Exception { + public void whenBulkSendSuccessfullyThenUpdateBatchItemStatus() { final List items = new ArrayList<>(); final String topicId = TestUtils.randomValidEventTypeName(); kafkaHelper.createTopic(topicId, ZOOKEEPER_URL); @@ -171,20 +217,6 @@ public void whenBulkSendSuccessfullyThenUpdateBatchItemStatus() throws Exception } } - @Test(timeout = 10000) - @SuppressWarnings("unchecked") - public void whenCreateTopicWithRetentionTime() throws Exception { - // ACT // - final String topicName = kafkaTopicRepository.createTopic(defaultTopicConfig); - - // ASSERT // - executeWithRetry(() -> Assert.assertEquals( - KafkaTestHelper.getTopicRetentionTime(topicName, ZOOKEEPER_URL), DEFAULT_RETENTION_TIME), - new RetryForSpecifiedTimeStrategy(5000) - .withExceptionsThatForceRetry(AssertionError.class) - .withWaitBetweenEachTry(500)); - } - private Map> getAllTopics() { final KafkaConsumer kafkaConsumer = kafkaHelper.createConsumer(); return kafkaConsumer.listTopics(); diff --git a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaTestHelper.java b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaTestHelper.java index 6bf1462b62..37704d2072 100644 --- a/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaTestHelper.java +++ b/src/acceptance-test/java/org/zalando/nakadi/repository/kafka/KafkaTestHelper.java @@ -117,7 +117,7 @@ public static String getTopicCleanupPolicy(final String topic, final String zkPa return getTopicProperty(topic, zkPath, "cleanup.policy"); } - private static String getTopicProperty(final String topic, final String zkPath, final String propertyName) { + public static String getTopicProperty(final String topic, final String zkPath, final String propertyName) { final ZkUtils zkUtils = ZkUtils.apply(zkPath, 30000, 10000, false); final Properties topicConfig = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic(), topic); return topicConfig.getProperty(propertyName); From 2d849d0c26a9c4a728355a8e78f2fe0a4407e4c1 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Thu, 12 Jul 2018 13:23:06 +0200 Subject: [PATCH 23/26] ARUHA-1757: changed default values; --- src/main/resources/application.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/resources/application.yml b/src/main/resources/application.yml index 7f0da14bc5..b71d903a24 100644 --- a/src/main/resources/application.yml +++ b/src/main/resources/application.yml @@ -49,8 +49,8 @@ nakadi: retentionMs: 172800000 # 2 days rotationMs: 86400000 # 1 day compacted: - rotationMs: 3600000 # 1 hour - segmentBytes: 104857600 # 100 MB + rotationMs: 86400000 # 1 hour + segmentBytes: 1073741824 # 1 GB compactionLagMs: 0 stream: timeoutMs: 31536000000 # 1 year :-P From ab385a54d0dd9b6acd0de774afbcc9d82944c7f4 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Thu, 12 Jul 2018 13:23:19 +0200 Subject: [PATCH 24/26] ARUHA-1757: changed default values; --- src/main/resources/application.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/resources/application.yml b/src/main/resources/application.yml index b71d903a24..0ccf08d2ab 100644 --- a/src/main/resources/application.yml +++ b/src/main/resources/application.yml @@ -49,7 +49,7 @@ nakadi: retentionMs: 172800000 # 2 days rotationMs: 86400000 # 1 day compacted: - rotationMs: 86400000 # 1 hour + rotationMs: 86400000 # 1 day segmentBytes: 1073741824 # 1 GB compactionLagMs: 0 stream: From 79ae03d7638ea70e1fae57b23dfad4e3f1ab3846 Mon Sep 17 00:00:00 2001 From: vstepanov Date: Thu, 12 Jul 2018 13:37:54 +0200 Subject: [PATCH 25/26] ARUHA-1757: changed default values; --- src/main/resources/application.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/resources/application.yml b/src/main/resources/application.yml index 0ccf08d2ab..8eac253139 100644 --- a/src/main/resources/application.yml +++ b/src/main/resources/application.yml @@ -51,7 +51,7 @@ nakadi: compacted: rotationMs: 86400000 # 1 day segmentBytes: 1073741824 # 1 GB - compactionLagMs: 0 + compactionLagMs: 10800000 # 3h stream: timeoutMs: 31536000000 # 1 year :-P default.commitTimeout: 60 # 1 minute From fe5b2aa61f8703baa70a31759bcaa9b63169565c Mon Sep 17 00:00:00 2001 From: vstepanov Date: Thu, 12 Jul 2018 15:31:53 +0200 Subject: [PATCH 26/26] ARUHA-1757: changed default values; --- src/main/resources/application.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/resources/application.yml b/src/main/resources/application.yml index 8eac253139..3ad5cb4d00 100644 --- a/src/main/resources/application.yml +++ b/src/main/resources/application.yml @@ -49,9 +49,9 @@ nakadi: retentionMs: 172800000 # 2 days rotationMs: 86400000 # 1 day compacted: - rotationMs: 86400000 # 1 day + rotationMs: 10800000 # 3 hours segmentBytes: 1073741824 # 1 GB - compactionLagMs: 10800000 # 3h + compactionLagMs: 10800000 # 3 hours stream: timeoutMs: 31536000000 # 1 year :-P default.commitTimeout: 60 # 1 minute