diff --git a/hazelcast-client/src/main/java/com/hazelcast/client/impl/clientside/ClientDynamicClusterConfig.java b/hazelcast-client/src/main/java/com/hazelcast/client/impl/clientside/ClientDynamicClusterConfig.java index bdb865be59f3a..7073670bd7c50 100644 --- a/hazelcast-client/src/main/java/com/hazelcast/client/impl/clientside/ClientDynamicClusterConfig.java +++ b/hazelcast-client/src/main/java/com/hazelcast/client/impl/clientside/ClientDynamicClusterConfig.java @@ -143,8 +143,9 @@ public Config addMapConfig(MapConfig mapConfig) { ClientMessage request = DynamicConfigAddMapConfigCodec.encodeRequest(mapConfig.getName(), mapConfig.getBackupCount(), mapConfig.getAsyncBackupCount(), mapConfig.getTimeToLiveSeconds(), mapConfig.getMaxIdleSeconds(), mapConfig.getEvictionPolicy().name(), mapConfig.isReadBackupData(), - mapConfig.getCacheDeserializedValues().name(), mapConfig.getMergePolicy(), mapConfig.getInMemoryFormat().name(), - listenerConfigs, partitionLostListenerConfigs, mapConfig.isStatisticsEnabled(), mapConfig.getQuorumName(), + mapConfig.getCacheDeserializedValues().name(), mapConfig.getMergePolicyConfig().getPolicy(), + mapConfig.getInMemoryFormat().name(), listenerConfigs, partitionLostListenerConfigs, + mapConfig.isStatisticsEnabled(), mapConfig.getQuorumName(), serializationService.toData(mapConfig.getMapEvictionPolicy()), mapConfig.getMaxSizeConfig().getMaxSizePolicy().name(), mapConfig.getMaxSizeConfig().getSize(), MapStoreConfigHolder.of(mapConfig.getMapStoreConfig(), serializationService), diff --git a/hazelcast-client/src/test/java/com/hazelcast/client/ClientListenersTest.java b/hazelcast-client/src/test/java/com/hazelcast/client/ClientListenersTest.java index fca1bbf4be8bf..562eae492e6ec 100644 --- a/hazelcast-client/src/test/java/com/hazelcast/client/ClientListenersTest.java +++ b/hazelcast-client/src/test/java/com/hazelcast/client/ClientListenersTest.java @@ -18,39 +18,39 @@ import com.hazelcast.client.config.ClientConfig; import com.hazelcast.client.test.TestHazelcastFactory; -import com.hazelcast.config.ListenerConfig; -import com.hazelcast.core.EntryAdapter; -import com.hazelcast.core.EntryEvent; -import com.hazelcast.core.EntryView; -import com.hazelcast.core.HazelcastInstance; import com.hazelcast.collection.IList; -import com.hazelcast.map.IMap; -import com.hazelcast.topic.ITopic; import com.hazelcast.collection.IQueue; import com.hazelcast.collection.ISet; import com.hazelcast.collection.ItemEvent; import com.hazelcast.collection.ItemListener; +import com.hazelcast.config.ListenerConfig; +import com.hazelcast.core.EntryAdapter; +import com.hazelcast.core.EntryEvent; +import com.hazelcast.core.HazelcastInstance; import com.hazelcast.core.LifecycleEvent; import com.hazelcast.core.LifecycleListener; -import com.hazelcast.topic.Message; -import com.hazelcast.topic.MessageListener; import com.hazelcast.instance.impl.Node; -import com.hazelcast.map.impl.EntryViews; +import com.hazelcast.map.IMap; import com.hazelcast.map.impl.MapService; -import com.hazelcast.map.impl.operation.LegacyMergeOperation; +import com.hazelcast.map.impl.operation.MergeOperation; import com.hazelcast.map.impl.record.Record; import com.hazelcast.map.listener.EntryMergedListener; -import com.hazelcast.map.merge.PassThroughMergePolicy; import com.hazelcast.nio.serialization.Data; import com.hazelcast.nio.serialization.Portable; import com.hazelcast.nio.serialization.PortableFactory; import com.hazelcast.spi.impl.NodeEngineImpl; +import com.hazelcast.spi.impl.operationservice.Operation; import com.hazelcast.spi.impl.operationservice.impl.OperationServiceImpl; +import com.hazelcast.spi.merge.PassThroughMergePolicy; +import com.hazelcast.spi.merge.SplitBrainMergeTypes; import com.hazelcast.spi.serialization.SerializationService; import com.hazelcast.test.HazelcastParallelClassRunner; import com.hazelcast.test.HazelcastTestSupport; import com.hazelcast.test.annotation.ParallelJVMTest; import com.hazelcast.test.annotation.QuickTest; +import com.hazelcast.topic.ITopic; +import com.hazelcast.topic.Message; +import com.hazelcast.topic.MessageListener; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -58,9 +58,11 @@ import org.junit.runner.RunWith; import org.mockito.Mockito; +import java.util.Collections; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import static com.hazelcast.spi.impl.merge.MergingValueFactory.createMergingEntry; import static org.junit.Assert.assertTrue; @RunWith(HazelcastParallelClassRunner.class) @@ -126,12 +128,13 @@ public void entryMerged(EntryEvent event) { Node node = getNode(server); NodeEngineImpl nodeEngine = node.nodeEngine; - OperationServiceImpl operationService = (OperationServiceImpl) nodeEngine.getOperationService(); + OperationServiceImpl operationService = nodeEngine.getOperationService(); SerializationService serializationService = getSerializationService(server); Data key = serializationService.toData(1); Data value = serializationService.toData(new ClientRegressionWithMockNetworkTest.SamplePortable(1)); - EntryView entryView = EntryViews.createSimpleEntryView(key, value, Mockito.mock(Record.class)); - LegacyMergeOperation op = new LegacyMergeOperation(map.getName(), entryView, new PassThroughMergePolicy(), false); + SplitBrainMergeTypes.MapMergeTypes mergingEntry = createMergingEntry(serializationService, key, value, Mockito.mock(Record.class)); + Operation op = new MergeOperation(map.getName(), Collections.singletonList(mergingEntry), + new PassThroughMergePolicy<>(), false); int partitionId = nodeEngine.getPartitionService().getPartitionId(key); operationService.invokeOnPartition(MapService.SERVICE_NAME, op, partitionId); diff --git a/hazelcast-client/src/test/java/com/hazelcast/client/map/ClientMapWANExceptionTest.java b/hazelcast-client/src/test/java/com/hazelcast/client/map/ClientMapWANExceptionTest.java index 99c935e9b3cdf..6f5d340311438 100644 --- a/hazelcast-client/src/test/java/com/hazelcast/client/map/ClientMapWANExceptionTest.java +++ b/hazelcast-client/src/test/java/com/hazelcast/client/map/ClientMapWANExceptionTest.java @@ -23,7 +23,7 @@ import com.hazelcast.config.WanReplicationRef; import com.hazelcast.core.HazelcastInstance; import com.hazelcast.map.IMap; -import com.hazelcast.map.merge.PassThroughMergePolicy; +import com.hazelcast.spi.merge.PassThroughMergePolicy; import com.hazelcast.test.HazelcastParallelClassRunner; import com.hazelcast.test.HazelcastTestSupport; import com.hazelcast.test.annotation.ParallelJVMTest; diff --git a/hazelcast-spring/pom.xml b/hazelcast-spring/pom.xml index a09b66537114e..ba9a72a1450b9 100644 --- a/hazelcast-spring/pom.xml +++ b/hazelcast-spring/pom.xml @@ -36,6 +36,7 @@ 4.3.23.RELEASE 1.0 3.8 + 1.3.2 @@ -148,7 +149,7 @@ com.hazelcast hazelcast-hibernate5 - 1.2.3 + ${hazelcast-hibernate5.version} test @@ -186,7 +187,7 @@ com.hazelcast hazelcast-hibernate3 - ${hazelcast.latest.version} + 3.8.4 test @@ -256,5 +257,11 @@ ${http.components.version} test + + com.hazelcast + hazelcast-hibernate5 + ${hazelcast-hibernate5.version} + test + diff --git a/hazelcast-spring/src/main/resources/hazelcast-spring-4.0.xsd b/hazelcast-spring/src/main/resources/hazelcast-spring-4.0.xsd index 97f7763ea363d..9f2728da680b2 100644 --- a/hazelcast-spring/src/main/resources/hazelcast-spring-4.0.xsd +++ b/hazelcast-spring/src/main/resources/hazelcast-spring-4.0.xsd @@ -3768,28 +3768,6 @@ Resolve conflicts that occurred when target cluster already has the replicated entry key. - - 4 merge policy implementations for IMap and 2 merge policy implementations for - ICache are provided out-of-the-box. - - IMap has the following merge policies: - com.hazelcast.map.merge.PutIfAbsentMapMergePolicy: Incoming entry merges from the - source map to the target map if it does not exist in the target map. - com.hazelcast.map.merge.HigherHitsMapMergePolicy: Incoming entry merges from the - source map to the target map if the source entry has more hits than the target one. - com.hazelcast.map.merge.PassThroughMergePolicy: Incoming entry merges from the - source map to the target map unless the incoming entry is not null. - com.hazelcast.map.merge.LatestUpdateMapMergePolicy: Incoming entry merges from the - source map to the target map if the source entry has been updated more recently - than the target entry. Please note that this merge policy can only be used when the - clusters' clocks are in sync. - - ICache has the following merge policies: - com.hazelcast.cache.merge.HigherHitsCacheMergePolicy: Incoming entry merges from - the source cache to the target cache if the source entry has more hits than the - target one. - com.hazelcast.cache.merge.PassThroughCacheMergePolicy: Incoming entry merges from - the source cache to the target cache unless the incoming entry is not null. diff --git a/hazelcast/src/main/java/com/hazelcast/cache/BuiltInCacheMergePolicies.java b/hazelcast/src/main/java/com/hazelcast/cache/BuiltInCacheMergePolicies.java deleted file mode 100644 index 5f2e429c08e5f..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/BuiltInCacheMergePolicies.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache; - -import com.hazelcast.cache.merge.HigherHitsCacheMergePolicy; -import com.hazelcast.cache.merge.LatestAccessCacheMergePolicy; -import com.hazelcast.cache.merge.PassThroughCacheMergePolicy; -import com.hazelcast.cache.merge.PutIfAbsentCacheMergePolicy; - -/** - * Enum that represents all built-in {@link com.hazelcast.cache.CacheMergePolicy} implementations. - *

- * Note: When a new built-in {@link com.hazelcast.cache.CacheMergePolicy} is implemented, - * its definition should be added here. - */ -public enum BuiltInCacheMergePolicies { - - /** - * Cache merge policy that merges cache entries from source to destination directly. - */ - PASS_THROUGH(PassThroughCacheMergePolicy.class, PassThroughCacheMergePolicy::new), - - /** - * Cache merge policy that merges cache entries from source to destination - * if they don't exist in the destination cache. - */ - PUT_IF_ABSENT(PutIfAbsentCacheMergePolicy.class, PutIfAbsentCacheMergePolicy::new), - - /** - * Cache merge policy that merges cache entries from source to destination cache - * if the source entry has more hits than the destination one. - */ - HIGHER_HITS(HigherHitsCacheMergePolicy.class, HigherHitsCacheMergePolicy::new), - - /** - * Cache merge policy that merges cache entries from source to destination cache - * if the source entry has been accessed more recently than the destination entry. - */ - LATEST_ACCESS(LatestAccessCacheMergePolicy.class, LatestAccessCacheMergePolicy::new); - - private Class implClass; - private CacheMergePolicyInstanceFactory instanceFactory; - - BuiltInCacheMergePolicies(Class implClass, CacheMergePolicyInstanceFactory instanceFactory) { - this.implClass = implClass; - this.instanceFactory = instanceFactory; - } - - public Class getImplementationClass() { - return implClass; - } - - /** - * Gets the name of the implementation class of {@link CacheMergePolicy}. - * - * @return the name of the implementation class of {@link CacheMergePolicy} - */ - public String getImplementationClassName() { - return implClass.getName(); - } - - /** - * Create a new instance of {@link CacheMergePolicy}. - * - * @return the created instance of {@link CacheMergePolicy} - */ - public CacheMergePolicy newInstance() { - return instanceFactory.create(); - } - - /** - * Gets the definition of the default {@link CacheMergePolicy}. - * - * @return the definition of the default {@link CacheMergePolicy} - */ - public static BuiltInCacheMergePolicies getDefault() { - return PUT_IF_ABSENT; - } - - private interface CacheMergePolicyInstanceFactory { - CacheMergePolicy create(); - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/cache/CacheMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/cache/CacheMergePolicy.java deleted file mode 100644 index 30424cb887c81..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/CacheMergePolicy.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache; - -import com.hazelcast.nio.serialization.BinaryInterface; - -import java.io.Serializable; - -/** - * Policy for merging cache entries after a split-brain has been healed. - *

- * Passed {@link CacheEntryView} instances wrap the key and value as their original types - * with conversion to object from their storage types. If you don't need the original types - * of key and value, you should use {@link StorageTypeAwareCacheMergePolicy} which is - * a sub-type of this interface. - * - * @see com.hazelcast.cache.merge.HigherHitsCacheMergePolicy - * @see com.hazelcast.cache.merge.LatestAccessCacheMergePolicy - * @see com.hazelcast.cache.merge.PassThroughCacheMergePolicy - * @see com.hazelcast.cache.merge.PutIfAbsentCacheMergePolicy - */ -@BinaryInterface -public interface CacheMergePolicy extends Serializable { - - /** - * Selects one of the merging and existing cache entries to be merged. - *

- * Note that the {@code existingEntry} may be {@code null} if there - * is no entry with the same key in the destination cache. - * This happens, when the entry for that key was - *

    - *
  • only created in the smaller sub-cluster during the split-brain
  • - *
  • removed in the larger sub-cluster during the split-brain
  • - *
- * - * @param cacheName name of the cache - * @param mergingEntry {@link CacheEntryView} instance that has the cache entry to be merged - * @param existingEntry {@link CacheEntryView} instance that has the existing cache entry - * or {@code null} if there is no existing cache entry - * @return the selected value for merging or {@code null} if the entry should be removed - */ - Object merge(String cacheName, CacheEntryView mergingEntry, CacheEntryView existingEntry); -} diff --git a/hazelcast/src/main/java/com/hazelcast/cache/StorageTypeAwareCacheMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/cache/StorageTypeAwareCacheMergePolicy.java deleted file mode 100644 index 93c53cebd49f5..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/StorageTypeAwareCacheMergePolicy.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache; - -/** - * Marker interface for indicating that key and value wrapped by - * {@link com.hazelcast.cache.CacheEntryView} will be not converted to their original types. - *

- * The motivation of this interface is that generally while merging cache entries, actual key and value are not checked. - * So there is no need to convert them to their original types. - *

- * In the worst case, the value is returned from the merge method as selected, which means that in all cases - * the value is accessed. So although the conversion is done lazily, it will be processed at this point. - * But by default, the key and value are converted to their original types - * unless this {@link com.hazelcast.cache.StorageTypeAwareCacheMergePolicy} is used. - *

- * Another motivation for using this interface is that at server side - * there is no need to locate classes of stored entries. - * It means that entries can be put from client with {@code BINARY} in-memory format and - * classpath of client can be different from server. - * So in this case, if entries are tried to convert to their original types while merging, - * {@link java.lang.ClassNotFoundException} is thrown here. - *

- * As a result, both of performance and {@link java.lang.ClassNotFoundException} as mentioned above, - * it is strongly recommended to use this interface if original values of key and values are not needed. - * - * @see com.hazelcast.cache.CacheMergePolicy - */ -public interface StorageTypeAwareCacheMergePolicy extends CacheMergePolicy { -} diff --git a/hazelcast/src/main/java/com/hazelcast/cache/impl/AbstractCacheRecordStore.java b/hazelcast/src/main/java/com/hazelcast/cache/impl/AbstractCacheRecordStore.java index 83a48807b86eb..3f155ef711291 100644 --- a/hazelcast/src/main/java/com/hazelcast/cache/impl/AbstractCacheRecordStore.java +++ b/hazelcast/src/main/java/com/hazelcast/cache/impl/AbstractCacheRecordStore.java @@ -16,13 +16,9 @@ package com.hazelcast.cache.impl; -import com.hazelcast.cache.CacheEntryView; import com.hazelcast.cache.CacheEventType; -import com.hazelcast.cache.CacheMergePolicy; import com.hazelcast.cache.CacheNotExistsException; -import com.hazelcast.cache.StorageTypeAwareCacheMergePolicy; import com.hazelcast.cache.impl.maxsize.impl.EntryCountCacheEvictionChecker; -import com.hazelcast.cache.impl.merge.entry.LazyCacheEntryView; import com.hazelcast.cache.impl.record.CacheRecord; import com.hazelcast.cache.impl.record.CacheRecordFactory; import com.hazelcast.cache.impl.record.SampleableCacheRecordMap; @@ -1776,69 +1772,6 @@ record = createRecordWithExpiry(key, newValue, expiryTime, now, disableWriteThro return merged ? record : null; } - @Override - public CacheRecord merge(CacheEntryView cacheEntryView, CacheMergePolicy mergePolicy, - String caller, String origin, int completionId, CallerProvenance callerProvenance) { - final long now = Clock.currentTimeMillis(); - final long start = isStatisticsEnabled() ? System.nanoTime() : 0; - - boolean merged = false; - Data key = cacheEntryView.getKey(); - Data value = cacheEntryView.getValue(); - long expiryTime = cacheEntryView.getExpirationTime(); - R record = records.get(key); - boolean isExpired = processExpiredEntry(key, record, now); - boolean disableWriteThrough = !persistenceEnabledFor(callerProvenance); - - if (record == null || isExpired) { - Object newValue = mergePolicy.merge(name, createCacheEntryView( - key, - value, - cacheEntryView.getCreationTime(), - cacheEntryView.getExpirationTime(), - cacheEntryView.getLastAccessTime(), - cacheEntryView.getAccessHit(), - cacheEntryView.getExpiryPolicy(), - mergePolicy), - null); - if (newValue != null) { - record = createRecordWithExpiry(key, newValue, expiryTime, now, disableWriteThrough, IGNORE_COMPLETION); - merged = record != null; - } - } else { - Object oldValue = record.getValue(); - Object newValue = mergePolicy.merge(name, - createCacheEntryView( - key, - value, - cacheEntryView.getCreationTime(), - cacheEntryView.getExpirationTime(), - cacheEntryView.getLastAccessTime(), - cacheEntryView.getAccessHit(), - cacheEntryView.getExpiryPolicy(), - mergePolicy), - createCacheEntryView( - key, - oldValue, - cacheEntryView.getCreationTime(), - record.getExpirationTime(), - record.getLastAccessTime(), - record.getAccessHit(), - record.getExpiryPolicy(), - mergePolicy)); - - - merged = updateWithMergingValue(key, oldValue, newValue, record, expiryTime, now, disableWriteThrough); - } - - if (merged && isStatisticsEnabled()) { - statistics.increaseCachePuts(1); - statistics.addPutTimeNanos(System.nanoTime() - start); - } - - return merged ? record : null; - } - private boolean updateWithMergingValue(Data key, Object existingValue, Object mergingValue, R record, long expiryTime, long now, boolean disableWriteThrough) { @@ -1859,15 +1792,6 @@ private Object getExpiryPolicyOrNull(R record) { return null; } - private CacheEntryView createCacheEntryView(Object key, Object value, long creationTime, long expirationTime, - long lastAccessTime, long accessHit, Object expiryPolicy, - CacheMergePolicy mergePolicy) { - // null serialization service means that use as storage type without conversion, - // non-null serialization service means that conversion is required - SerializationService ss = mergePolicy instanceof StorageTypeAwareCacheMergePolicy ? null : this.ss; - return new LazyCacheEntryView(key, value, creationTime, expirationTime, lastAccessTime, accessHit, expiryPolicy, ss); - } - @Override public CacheKeyIterationResult fetchKeys(int tableIndex, int size) { return records.fetchKeys(tableIndex, size); diff --git a/hazelcast/src/main/java/com/hazelcast/cache/impl/AbstractCacheService.java b/hazelcast/src/main/java/com/hazelcast/cache/impl/AbstractCacheService.java index 74cb24ef2d920..160e06b346bf6 100644 --- a/hazelcast/src/main/java/com/hazelcast/cache/impl/AbstractCacheService.java +++ b/hazelcast/src/main/java/com/hazelcast/cache/impl/AbstractCacheService.java @@ -22,18 +22,17 @@ import com.hazelcast.cache.impl.eviction.CacheClearExpiredRecordsTask; import com.hazelcast.cache.impl.journal.CacheEventJournal; import com.hazelcast.cache.impl.journal.RingbufferCacheEventJournalImpl; -import com.hazelcast.cache.impl.merge.policy.CacheMergePolicyProvider; import com.hazelcast.cache.impl.operation.AddCacheConfigOperationSupplier; import com.hazelcast.cache.impl.operation.OnJoinCacheOperation; import com.hazelcast.cache.impl.tenantcontrol.CacheDestroyEventContext; import com.hazelcast.cluster.ClusterState; +import com.hazelcast.cluster.Member; import com.hazelcast.config.CacheConfig; import com.hazelcast.config.CacheConfigAccessor; import com.hazelcast.config.CacheSimpleConfig; import com.hazelcast.config.InMemoryFormat; import com.hazelcast.core.DistributedObject; import com.hazelcast.core.ICompletableFuture; -import com.hazelcast.cluster.Member; import com.hazelcast.internal.cluster.ClusterStateListener; import com.hazelcast.internal.eviction.ExpirationManager; import com.hazelcast.internal.util.InvocationUtil; @@ -45,14 +44,15 @@ import com.hazelcast.spi.EventRegistration; import com.hazelcast.spi.EventService; import com.hazelcast.spi.NodeEngine; -import com.hazelcast.spi.partition.PartitionAwareService; -import com.hazelcast.spi.partition.PartitionMigrationEvent; -import com.hazelcast.spi.impl.operationservice.Operation; import com.hazelcast.spi.PreJoinAwareService; import com.hazelcast.spi.QuorumAwareService; import com.hazelcast.spi.SplitBrainHandlerService; +import com.hazelcast.spi.impl.operationservice.Operation; +import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; import com.hazelcast.spi.partition.IPartitionLostEvent; import com.hazelcast.spi.partition.MigrationEndpoint; +import com.hazelcast.spi.partition.PartitionAwareService; +import com.hazelcast.spi.partition.PartitionMigrationEvent; import com.hazelcast.spi.tenantcontrol.TenantControlFactory; import com.hazelcast.util.Clock; import com.hazelcast.util.ConcurrencyUtil; @@ -122,8 +122,8 @@ public abstract class AbstractCacheService implements ICacheService, PreJoinAwar protected final ConstructorFunction cacheContextsConstructorFunction = name -> new CacheContext(); protected final ConstructorFunction cacheStatisticsConstructorFunction = name -> new CacheStatisticsImpl( - Clock.currentTimeMillis(), - CacheEntryCountResolver.createEntryCountResolver(getOrCreateCacheContext(name))); + Clock.currentTimeMillis(), + CacheEntryCountResolver.createEntryCountResolver(getOrCreateCacheContext(name))); protected final ConstructorFunction> cacheResourcesConstructorFunction = name -> newSetFromMap(new ConcurrentHashMap()); @@ -136,7 +136,7 @@ public abstract class AbstractCacheService implements ICacheService, PreJoinAwar protected CachePartitionSegment[] segments; protected CacheEventHandler cacheEventHandler; protected RingbufferCacheEventJournalImpl eventJournal; - protected CacheMergePolicyProvider mergePolicyProvider; + protected SplitBrainMergePolicyProvider mergePolicyProvider; protected CacheSplitBrainHandlerService splitBrainHandlerService; protected CacheClearExpiredRecordsTask clearExpiredRecordsTask; protected ExpirationManager expirationManager; @@ -155,12 +155,12 @@ public final void init(NodeEngine nodeEngine, Properties properties) { this.splitBrainHandlerService = new CacheSplitBrainHandlerService(nodeEngine, segments); this.logger = nodeEngine.getLogger(getClass()); this.eventJournal = new RingbufferCacheEventJournalImpl(nodeEngine); - this.mergePolicyProvider = new CacheMergePolicyProvider(nodeEngine); + this.mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); postInit(nodeEngine, properties); } - public CacheMergePolicyProvider getMergePolicyProvider() { + public SplitBrainMergePolicyProvider getMergePolicyProvider() { return mergePolicyProvider; } @@ -252,6 +252,7 @@ public DistributedObject createDistributedObject(String cacheNameWithPrefix) { cacheConfig.setManagerPrefix(HazelcastCacheManager.CACHE_MANAGER_PREFIX); } + SplitBrainMergePolicyProvider mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); checkCacheConfig(cacheConfig, mergePolicyProvider); Object mergePolicy = mergePolicyProvider.getMergePolicy(cacheConfig.getMergePolicy()); @@ -677,13 +678,10 @@ public void addCacheResource(String cacheNameWithPrefix, Closeable resource) { protected void deleteCacheResources(String name) { Set cacheResources; - ContextMutexFactory.Mutex mutex = cacheResourcesMutexFactory.mutexFor(name); - try { + try (ContextMutexFactory.Mutex mutex = cacheResourcesMutexFactory.mutexFor(name)) { synchronized (mutex) { cacheResources = resources.remove(name); } - } finally { - mutex.close(); } if (cacheResources != null) { diff --git a/hazelcast/src/main/java/com/hazelcast/cache/impl/CacheDataSerializerHook.java b/hazelcast/src/main/java/com/hazelcast/cache/impl/CacheDataSerializerHook.java index 6e80c9faf0925..a960100944afb 100644 --- a/hazelcast/src/main/java/com/hazelcast/cache/impl/CacheDataSerializerHook.java +++ b/hazelcast/src/main/java/com/hazelcast/cache/impl/CacheDataSerializerHook.java @@ -43,7 +43,6 @@ import com.hazelcast.cache.impl.operation.CacheGetInvalidationMetaDataOperation; import com.hazelcast.cache.impl.operation.CacheGetOperation; import com.hazelcast.cache.impl.operation.CacheKeyIteratorOperation; -import com.hazelcast.cache.impl.operation.CacheLegacyMergeOperation; import com.hazelcast.cache.impl.operation.CacheListenerRegistrationOperation; import com.hazelcast.cache.impl.operation.CacheLoadAllOperation; import com.hazelcast.cache.impl.operation.CacheLoadAllOperationFactory; @@ -131,41 +130,38 @@ public final class CacheDataSerializerHook public static final short REMOVE_ALL_BACKUP = 35; public static final short REMOVE_ALL_FACTORY = 36; public static final short PUT_ALL = 37; - public static final short LEGACY_MERGE = 38; - public static final short INVALIDATION_MESSAGE = 39; - public static final short BATCH_INVALIDATION_MESSAGE = 40; - public static final short ENTRY_ITERATOR = 41; - public static final short ENTRY_ITERATION_RESULT = 42; - public static final short CACHE_PARTITION_LOST_EVENT_FILTER = 43; - public static final short DEFAULT_CACHE_ENTRY_VIEW = 44; - public static final short CACHE_REPLICATION = 45; - public static final short CACHE_POST_JOIN = 46; - public static final short CACHE_DATA_RECORD = 47; - public static final short CACHE_OBJECT_RECORD = 48; - public static final short CACHE_PARTITION_EVENT_DATA = 49; + public static final short ENTRY_ITERATOR = 38; + public static final short ENTRY_ITERATION_RESULT = 39; + public static final short CACHE_PARTITION_LOST_EVENT_FILTER = 40; + public static final short DEFAULT_CACHE_ENTRY_VIEW = 41; + public static final short CACHE_REPLICATION = 42; + public static final short CACHE_POST_JOIN = 43; + public static final short CACHE_DATA_RECORD = 44; + public static final short CACHE_OBJECT_RECORD = 45; + public static final short CACHE_PARTITION_EVENT_DATA = 46; - public static final short CACHE_INVALIDATION_METADATA = 50; - public static final short CACHE_INVALIDATION_METADATA_RESPONSE = 51; - public static final short CACHE_ASSIGN_AND_GET_UUIDS = 52; - public static final short CACHE_ASSIGN_AND_GET_UUIDS_FACTORY = 53; - public static final short CACHE_NEAR_CACHE_STATE_HOLDER = 54; - public static final short CACHE_EVENT_LISTENER_ADAPTOR = 55; - public static final short EVENT_JOURNAL_SUBSCRIBE_OPERATION = 56; - public static final short EVENT_JOURNAL_READ_OPERATION = 57; - public static final short EVENT_JOURNAL_DESERIALIZING_CACHE_EVENT = 58; - public static final short EVENT_JOURNAL_INTERNAL_CACHE_EVENT = 59; - public static final short EVENT_JOURNAL_READ_RESULT_SET = 60; - public static final int PRE_JOIN_CACHE_CONFIG = 61; - public static final int CACHE_BROWSER_ENTRY_VIEW = 62; - public static final int GET_CACHE_ENTRY_VIEW_PROCESSOR = 63; + public static final short CACHE_INVALIDATION_METADATA = 47; + public static final short CACHE_INVALIDATION_METADATA_RESPONSE = 48; + public static final short CACHE_ASSIGN_AND_GET_UUIDS = 49; + public static final short CACHE_ASSIGN_AND_GET_UUIDS_FACTORY = 50; + public static final short CACHE_NEAR_CACHE_STATE_HOLDER = 51; + public static final short CACHE_EVENT_LISTENER_ADAPTOR = 52; + public static final short EVENT_JOURNAL_SUBSCRIBE_OPERATION = 53; + public static final short EVENT_JOURNAL_READ_OPERATION = 54; + public static final short EVENT_JOURNAL_DESERIALIZING_CACHE_EVENT = 55; + public static final short EVENT_JOURNAL_INTERNAL_CACHE_EVENT = 56; + public static final short EVENT_JOURNAL_READ_RESULT_SET = 57; + public static final int PRE_JOIN_CACHE_CONFIG = 58; + public static final int CACHE_BROWSER_ENTRY_VIEW = 59; + public static final int GET_CACHE_ENTRY_VIEW_PROCESSOR = 60; - public static final int MERGE_FACTORY = 64; - public static final int MERGE = 65; - public static final int ADD_CACHE_CONFIG_OPERATION = 66; - public static final int SET_EXPIRY_POLICY = 67; - public static final int SET_EXPIRY_POLICY_BACKUP = 68; - public static final int EXPIRE_BATCH_BACKUP = 69; - public static final int CACHE_DESTROY_EVENT_CONTEXT = 70; + public static final int MERGE_FACTORY = 61; + public static final int MERGE = 62; + public static final int ADD_CACHE_CONFIG_OPERATION = 63; + public static final int SET_EXPIRY_POLICY = 64; + public static final int SET_EXPIRY_POLICY_BACKUP = 65; + public static final int EXPIRE_BATCH_BACKUP = 66; + public static final int CACHE_DESTROY_EVENT_CONTEXT = 67; private static final int LEN = CACHE_DESTROY_EVENT_CONTEXT + 1; @@ -212,7 +208,6 @@ public DataSerializableFactory createFactory() { constructors[REMOVE_ALL_BACKUP] = arg -> new CacheRemoveAllBackupOperation(); constructors[REMOVE_ALL_FACTORY] = arg -> new CacheRemoveAllOperationFactory(); constructors[PUT_ALL] = arg -> new CachePutAllOperation(); - constructors[LEGACY_MERGE] = arg -> new CacheLegacyMergeOperation(); constructors[ENTRY_ITERATOR] = arg -> new CacheEntryIteratorOperation(); constructors[ENTRY_ITERATION_RESULT] = arg -> new CacheEntryIterationResult(); constructors[CACHE_PARTITION_LOST_EVENT_FILTER] = arg -> new CachePartitionLostEventFilter(); diff --git a/hazelcast/src/main/java/com/hazelcast/cache/impl/CacheMergeRunnable.java b/hazelcast/src/main/java/com/hazelcast/cache/impl/CacheMergeRunnable.java index 7a0a4bd0e7fd4..3aad09fb831c4 100644 --- a/hazelcast/src/main/java/com/hazelcast/cache/impl/CacheMergeRunnable.java +++ b/hazelcast/src/main/java/com/hazelcast/cache/impl/CacheMergeRunnable.java @@ -16,18 +16,13 @@ package com.hazelcast.cache.impl; -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.CacheMergePolicy; -import com.hazelcast.cache.impl.merge.entry.DefaultCacheEntryView; -import com.hazelcast.cache.impl.operation.CacheLegacyMergeOperation; import com.hazelcast.cache.impl.record.CacheRecord; import com.hazelcast.config.CacheConfig; import com.hazelcast.config.InMemoryFormat; import com.hazelcast.nio.serialization.Data; import com.hazelcast.spi.NodeEngine; -import com.hazelcast.spi.impl.operationservice.Operation; -import com.hazelcast.spi.impl.operationservice.OperationFactory; import com.hazelcast.spi.impl.merge.AbstractMergeRunnable; +import com.hazelcast.spi.impl.operationservice.OperationFactory; import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergeTypes.CacheMergeTypes; @@ -84,28 +79,6 @@ protected void mergeStore(ICacheRecordStore store, BiConsumer consumer) { - int partitionId = recordStore.getPartitionId(); - String name = recordStore.getName(); - CacheMergePolicy mergePolicy = ((CacheMergePolicy) getMergePolicy(name)); - - for (Map.Entry entry : recordStore.getReadOnlyRecords().entrySet()) { - Data key = entry.getKey(); - CacheRecord record = entry.getValue(); - CacheEntryView entryView = new DefaultCacheEntryView( - key, - toData(record.getValue()), - record.getCreationTime(), - record.getExpirationTime(), - record.getLastAccessTime(), - record.getAccessHit(), - toData(record.getExpiryPolicy())); - - consumer.accept(partitionId, new CacheLegacyMergeOperation(name, key, entryView, mergePolicy)); - } - } - @Override protected InMemoryFormat getInMemoryFormat(String dataStructureName) { return cacheService.getConfigs().get(dataStructureName).getInMemoryFormat(); @@ -120,8 +93,8 @@ protected int getBatchSize(String dataStructureName) { } @Override - protected Object getMergePolicy(String dataStructureName) { - return cacheService.getMergePolicy(dataStructureName); + protected SplitBrainMergePolicy getMergePolicy(String dataStructureName) { + return mergePolicyProvider.getMergePolicy(dataStructureName); } @Override diff --git a/hazelcast/src/main/java/com/hazelcast/cache/impl/HazelcastServerCacheManager.java b/hazelcast/src/main/java/com/hazelcast/cache/impl/HazelcastServerCacheManager.java index a286a6733cd6a..0b33cb986781a 100644 --- a/hazelcast/src/main/java/com/hazelcast/cache/impl/HazelcastServerCacheManager.java +++ b/hazelcast/src/main/java/com/hazelcast/cache/impl/HazelcastServerCacheManager.java @@ -18,18 +18,18 @@ import com.hazelcast.cache.HazelcastCacheManager; import com.hazelcast.cache.HazelcastCachingProvider; -import com.hazelcast.cache.impl.merge.policy.CacheMergePolicyProvider; import com.hazelcast.cache.impl.operation.CacheGetConfigOperation; import com.hazelcast.cache.impl.operation.CacheManagementConfigOperation; +import com.hazelcast.cluster.Member; import com.hazelcast.config.CacheConfig; import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.cluster.Member; import com.hazelcast.instance.HazelcastInstanceCacheManager; import com.hazelcast.instance.impl.HazelcastInstanceImpl; import com.hazelcast.instance.impl.HazelcastInstanceProxy; import com.hazelcast.spi.InternalCompletableFuture; import com.hazelcast.spi.NodeEngine; import com.hazelcast.spi.impl.operationservice.OperationService; +import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; import java.net.URI; import java.util.ArrayList; @@ -179,7 +179,7 @@ protected void removeCacheConfigFromLocal(String cacheNameWithPrefix) { @Override protected void validateCacheConfig(CacheConfig cacheConfig) { - CacheMergePolicyProvider mergePolicyProvider = cacheService.getMergePolicyProvider(); + SplitBrainMergePolicyProvider mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); checkCacheConfig(cacheConfig, mergePolicyProvider); Object mergePolicy = mergePolicyProvider.getMergePolicy(cacheConfig.getMergePolicy()); diff --git a/hazelcast/src/main/java/com/hazelcast/cache/impl/ICacheRecordStore.java b/hazelcast/src/main/java/com/hazelcast/cache/impl/ICacheRecordStore.java index 70469762cd8bf..8c22c7da586f8 100644 --- a/hazelcast/src/main/java/com/hazelcast/cache/impl/ICacheRecordStore.java +++ b/hazelcast/src/main/java/com/hazelcast/cache/impl/ICacheRecordStore.java @@ -16,8 +16,6 @@ package com.hazelcast.cache.impl; -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.CacheMergePolicy; import com.hazelcast.cache.impl.record.CacheRecord; import com.hazelcast.config.CacheConfig; import com.hazelcast.internal.eviction.ExpiredKey; @@ -526,21 +524,6 @@ public interface ICacheRecordStore { CacheRecord merge(CacheMergeTypes mergingEntry, SplitBrainMergePolicy mergePolicy, CallerProvenance callerProvenance); - /** - * Merges the given {@link CacheEntryView} via the given {@link CacheMergePolicy}. - * - * @param cacheEntryView the {@link CacheEntryView} instance to merge - * @param mergePolicy the {@link CacheMergePolicy} instance to apply - * @param caller the UUID of the caller - * @param origin source of the call - * @param completionId User generated id which shall be received as a field of the cache event upon completion of - * the request in the cluster. - * @param callerProvenance - * @return the used {@link CacheRecord} if merge is applied, otherwise {@code null} - */ - CacheRecord merge(CacheEntryView cacheEntryView, CacheMergePolicy mergePolicy, - String caller, String origin, int completionId, CallerProvenance callerProvenance); - /** * @return partition ID of this store */ diff --git a/hazelcast/src/main/java/com/hazelcast/cache/impl/merge/policy/CacheMergePolicyProvider.java b/hazelcast/src/main/java/com/hazelcast/cache/impl/merge/policy/CacheMergePolicyProvider.java deleted file mode 100644 index 5297eef06adb5..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/impl/merge/policy/CacheMergePolicyProvider.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.impl.merge.policy; - -import com.hazelcast.cache.BuiltInCacheMergePolicies; -import com.hazelcast.cache.CacheMergePolicy; -import com.hazelcast.config.InvalidConfigurationException; -import com.hazelcast.spi.NodeEngine; -import com.hazelcast.spi.merge.SplitBrainMergePolicy; -import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; -import com.hazelcast.util.ConstructorFunction; - -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import static com.hazelcast.nio.ClassLoaderUtil.newInstance; -import static com.hazelcast.util.ConcurrencyUtil.getOrPutIfAbsent; - -/** - * A provider for {@link com.hazelcast.cache.CacheMergePolicy} instances. - */ -public final class CacheMergePolicyProvider { - - private final ConcurrentMap mergePolicyMap = new ConcurrentHashMap(); - - private final ConstructorFunction policyConstructorFunction - = new ConstructorFunction() { - @Override - public CacheMergePolicy createNew(String className) { - try { - return newInstance(nodeEngine.getConfigClassLoader(), className); - } catch (Exception e) { - nodeEngine.getLogger(getClass()).severe(e); - throw new InvalidConfigurationException("Invalid cache merge policy: " + className, e); - } - } - }; - - private final NodeEngine nodeEngine; - private final SplitBrainMergePolicyProvider policyProvider; - - public CacheMergePolicyProvider(NodeEngine nodeEngine) { - this.nodeEngine = nodeEngine; - this.policyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); - addOutOfBoxPolicies(); - } - - private void addOutOfBoxPolicies() { - for (BuiltInCacheMergePolicies mergePolicy : BuiltInCacheMergePolicies.values()) { - CacheMergePolicy cacheMergePolicy = mergePolicy.newInstance(); - // register `CacheMergePolicy` by its constant - mergePolicyMap.put(mergePolicy.name(), cacheMergePolicy); - // register `CacheMergePolicy` by its name - mergePolicyMap.put(mergePolicy.getImplementationClassName(), cacheMergePolicy); - } - } - - /** - * Returns an instance of a merge policy by its classname. - *

- * First tries to resolve the classname as {@link SplitBrainMergePolicy}, - * then as {@link com.hazelcast.cache.CacheMergePolicy}. - *

- * If no merge policy matches an {@link InvalidConfigurationException} is thrown. - * - * @param className the classname of the given merge policy - * @return an instance of the merge policy class - * @throws InvalidConfigurationException if no matching merge policy class was found - */ - public Object getMergePolicy(String className) { - if (className == null) { - throw new InvalidConfigurationException("Class name is mandatory!"); - } - try { - return policyProvider.getMergePolicy(className); - } catch (InvalidConfigurationException e) { - return getOrPutIfAbsent(mergePolicyMap, className, policyConstructorFunction); - } - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/cache/impl/merge/policy/package-info.java b/hazelcast/src/main/java/com/hazelcast/cache/impl/merge/policy/package-info.java deleted file mode 100644 index d23c605d49d56..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/impl/merge/policy/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - *

This package contains merge policy internal interfaces and functionality for Hazelcast cache.
- */ -package com.hazelcast.cache.impl.merge.policy; diff --git a/hazelcast/src/main/java/com/hazelcast/cache/impl/operation/CacheLegacyMergeOperation.java b/hazelcast/src/main/java/com/hazelcast/cache/impl/operation/CacheLegacyMergeOperation.java deleted file mode 100644 index fe559927695a1..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/impl/operation/CacheLegacyMergeOperation.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.impl.operation; - -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.CacheMergePolicy; -import com.hazelcast.cache.impl.CacheDataSerializerHook; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.Data; -import com.hazelcast.spi.impl.operationservice.BackupAwareOperation; -import com.hazelcast.spi.impl.operationservice.Operation; - -import java.io.IOException; - -import static com.hazelcast.cache.impl.AbstractCacheRecordStore.SOURCE_NOT_AVAILABLE; -import static com.hazelcast.cache.impl.operation.MutableOperation.IGNORE_COMPLETION; -import static com.hazelcast.wan.impl.CallerProvenance.NOT_WAN; - -/** - * Contains a merging entry for split-brain healing with a {@link CacheMergePolicy}. - */ -public class CacheLegacyMergeOperation - extends KeyBasedCacheOperation - implements BackupAwareOperation { - - private CacheMergePolicy mergePolicy; - private CacheEntryView mergingEntry; - - public CacheLegacyMergeOperation() { - } - - public CacheLegacyMergeOperation(String name, Data key, - CacheEntryView entryView, CacheMergePolicy policy) { - super(name, key); - mergingEntry = entryView; - mergePolicy = policy; - } - - @Override - public void run() throws Exception { - backupRecord = recordStore.merge(mergingEntry, mergePolicy, - SOURCE_NOT_AVAILABLE, null, IGNORE_COMPLETION, NOT_WAN); - } - - @Override - public void afterRun() { - if (backupRecord != null) { - publishWanUpdate(key, backupRecord); - } - } - - @Override - public boolean shouldBackup() { - return backupRecord != null; - } - - @Override - public Operation getBackupOperation() { - return new CachePutBackupOperation(name, key, backupRecord); - } - - @Override - protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); - out.writeObject(mergingEntry); - out.writeObject(mergePolicy); - } - - @Override - protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); - mergingEntry = in.readObject(); - mergePolicy = in.readObject(); - } - - @Override - public int getClassId() { - return CacheDataSerializerHook.LEGACY_MERGE; - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/cache/merge/HigherHitsCacheMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/cache/merge/HigherHitsCacheMergePolicy.java deleted file mode 100644 index 3df5976ddf3fb..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/merge/HigherHitsCacheMergePolicy.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.StorageTypeAwareCacheMergePolicy; -import com.hazelcast.nio.serialization.BinaryInterface; - -/** - * Merges cache entries from source to destination cache if the source entry - * has more hits than the destination one. - */ -@BinaryInterface -public class HigherHitsCacheMergePolicy implements StorageTypeAwareCacheMergePolicy { - - public HigherHitsCacheMergePolicy() { - } - - @Override - public Object merge(String cacheName, CacheEntryView mergingEntry, CacheEntryView existingEntry) { - if (existingEntry == null || mergingEntry.getAccessHit() >= existingEntry.getAccessHit()) { - return mergingEntry.getValue(); - } - return existingEntry.getValue(); - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/cache/merge/LatestAccessCacheMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/cache/merge/LatestAccessCacheMergePolicy.java deleted file mode 100644 index d2733b97d2deb..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/merge/LatestAccessCacheMergePolicy.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.StorageTypeAwareCacheMergePolicy; -import com.hazelcast.nio.serialization.BinaryInterface; - -/** - * Merges cache entries from source to destination cache if the source entry - * has been accessed more recently than the destination entry. - *

- * Note: This policy can only be used if the clocks of the nodes are in sync. - */ -@BinaryInterface -public class LatestAccessCacheMergePolicy implements StorageTypeAwareCacheMergePolicy { - - public LatestAccessCacheMergePolicy() { - } - - @Override - public Object merge(String cacheName, CacheEntryView mergingEntry, CacheEntryView existingEntry) { - if (existingEntry == null || mergingEntry.getLastAccessTime() >= existingEntry.getLastAccessTime()) { - return mergingEntry.getValue(); - } - return existingEntry.getValue(); - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/cache/merge/PassThroughCacheMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/cache/merge/PassThroughCacheMergePolicy.java deleted file mode 100644 index 96781253cde8d..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/merge/PassThroughCacheMergePolicy.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.StorageTypeAwareCacheMergePolicy; -import com.hazelcast.nio.serialization.BinaryInterface; - -/** - * Merges cache entries from source to destination directly unless the merging entry is {@code null}. - */ -@BinaryInterface -public class PassThroughCacheMergePolicy implements StorageTypeAwareCacheMergePolicy { - - public PassThroughCacheMergePolicy() { - } - - @Override - public Object merge(String cacheName, CacheEntryView mergingEntry, CacheEntryView existingEntry) { - return mergingEntry != null ? mergingEntry.getValue() : existingEntry.getValue(); - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/cache/merge/PutIfAbsentCacheMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/cache/merge/PutIfAbsentCacheMergePolicy.java deleted file mode 100644 index 4cd99748b5286..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/merge/PutIfAbsentCacheMergePolicy.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.StorageTypeAwareCacheMergePolicy; -import com.hazelcast.nio.serialization.BinaryInterface; - -/** - * Merges cache entries from source to destination if they don't exist in the destination cache. - */ -@BinaryInterface -public class PutIfAbsentCacheMergePolicy implements StorageTypeAwareCacheMergePolicy { - - public PutIfAbsentCacheMergePolicy() { - } - - @Override - public Object merge(String cacheName, CacheEntryView mergingEntry, CacheEntryView existingEntry) { - if (existingEntry == null) { - return mergingEntry.getValue(); - } - return existingEntry.getValue(); - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/cache/merge/package-info.java b/hazelcast/src/main/java/com/hazelcast/cache/merge/package-info.java deleted file mode 100644 index 1be2560ddccd5..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/cache/merge/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Contains out-of-the-box merge policies for {@link com.hazelcast.cache.ICache}. - */ -package com.hazelcast.cache.merge; diff --git a/hazelcast/src/main/java/com/hazelcast/client/impl/protocol/task/cache/CacheCreateConfigMessageTask.java b/hazelcast/src/main/java/com/hazelcast/client/impl/protocol/task/cache/CacheCreateConfigMessageTask.java index 63ada9c73354a..5860633986ded 100644 --- a/hazelcast/src/main/java/com/hazelcast/client/impl/protocol/task/cache/CacheCreateConfigMessageTask.java +++ b/hazelcast/src/main/java/com/hazelcast/client/impl/protocol/task/cache/CacheCreateConfigMessageTask.java @@ -19,7 +19,6 @@ import com.hazelcast.cache.impl.CacheService; import com.hazelcast.cache.impl.ICacheService; import com.hazelcast.cache.impl.PreJoinCacheConfig; -import com.hazelcast.cache.impl.merge.policy.CacheMergePolicyProvider; import com.hazelcast.client.impl.protocol.ClientMessage; import com.hazelcast.client.impl.protocol.codec.CacheCreateConfigCodec; import com.hazelcast.client.impl.protocol.task.AbstractMessageTask; @@ -29,6 +28,7 @@ import com.hazelcast.instance.impl.Node; import com.hazelcast.nio.Connection; import com.hazelcast.nio.serialization.Data; +import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; import java.security.Permission; @@ -54,7 +54,7 @@ protected void processMessage() { CacheService cacheService = getService(CacheService.SERVICE_NAME); if (cacheConfig != null) { - CacheMergePolicyProvider mergePolicyProvider = cacheService.getMergePolicyProvider(); + SplitBrainMergePolicyProvider mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); checkCacheConfig(cacheConfig, mergePolicyProvider); Object mergePolicy = mergePolicyProvider.getMergePolicy(cacheConfig.getMergePolicy()); diff --git a/hazelcast/src/main/java/com/hazelcast/config/MapConfig.java b/hazelcast/src/main/java/com/hazelcast/config/MapConfig.java index be53fc344a828..850c5bd61d9b0 100644 --- a/hazelcast/src/main/java/com/hazelcast/config/MapConfig.java +++ b/hazelcast/src/main/java/com/hazelcast/config/MapConfig.java @@ -18,11 +18,9 @@ import com.hazelcast.map.IMap; import com.hazelcast.map.eviction.MapEvictionPolicy; -import com.hazelcast.map.merge.PutIfAbsentMapMergePolicy; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergeTypeProvider; import com.hazelcast.spi.merge.SplitBrainMergeTypes; import com.hazelcast.spi.partition.IPartition; @@ -92,10 +90,6 @@ public class MapConfig implements SplitBrainMergeTypeProvider, IdentifiedDataSer */ public static final EvictionPolicy DEFAULT_EVICTION_POLICY = EvictionPolicy.NONE; - /** - * Default policy for merging. - */ - public static final String DEFAULT_MAP_MERGE_POLICY = PutIfAbsentMapMergePolicy.class.getName(); /** * Default In-Memory format is binary. */ @@ -535,31 +529,6 @@ public MapConfig setNearCacheConfig(NearCacheConfig nearCacheConfig) { return this; } - /** - * Gets the merge policy. - * - * @return the merge policy classname - * @deprecated since 3.10, please use {@link #getMergePolicyConfig()} and {@link MergePolicyConfig#getPolicy()} - */ - public String getMergePolicy() { - return mergePolicyConfig.getPolicy(); - } - - /** - * Sets the merge policy. - *

- * Accepts a classname of {@link SplitBrainMergePolicy} - * or the deprecated {@link com.hazelcast.map.merge.MapMergePolicy}. - * - * @param mergePolicy the merge policy classname to set - * @return the updated map configuration - * @deprecated since 3.10, please use {@link #setMergePolicyConfig(MergePolicyConfig)} - */ - public MapConfig setMergePolicy(String mergePolicy) { - this.mergePolicyConfig.setPolicy(mergePolicy); - return this; - } - /** * Gets the {@link MergePolicyConfig} for this map. * diff --git a/hazelcast/src/main/java/com/hazelcast/config/MapConfigReadOnly.java b/hazelcast/src/main/java/com/hazelcast/config/MapConfigReadOnly.java index ce9f8900f417d..a36feaef9066c 100644 --- a/hazelcast/src/main/java/com/hazelcast/config/MapConfigReadOnly.java +++ b/hazelcast/src/main/java/com/hazelcast/config/MapConfigReadOnly.java @@ -222,11 +222,6 @@ public MapConfig setNearCacheConfig(NearCacheConfig nearCacheConfig) { throw throwReadOnly(); } - @Override - public MapConfig setMergePolicy(String mergePolicy) { - throw throwReadOnly(); - } - @Override public MapConfig setMergePolicyConfig(MergePolicyConfig mergePolicyConfig) { throw throwReadOnly(); diff --git a/hazelcast/src/main/java/com/hazelcast/config/ReplicatedMapConfig.java b/hazelcast/src/main/java/com/hazelcast/config/ReplicatedMapConfig.java index 342583951fc0b..f78fa59bdb977 100644 --- a/hazelcast/src/main/java/com/hazelcast/config/ReplicatedMapConfig.java +++ b/hazelcast/src/main/java/com/hazelcast/config/ReplicatedMapConfig.java @@ -337,28 +337,6 @@ public ReplicatedMapConfig setQuorumName(String quorumName) { return this; } - /** - * Gets the replicated map merge policy {@link com.hazelcast.replicatedmap.merge.ReplicatedMapMergePolicy} - * - * @return the updated replicated map configuration - * @deprecated since 3.10, please use {@link #getMergePolicyConfig()} and {@link MergePolicyConfig#getPolicy()} - */ - public String getMergePolicy() { - return mergePolicyConfig.getPolicy(); - } - - /** - * Sets the replicated map merge policy {@link com.hazelcast.replicatedmap.merge.ReplicatedMapMergePolicy} - * - * @param mergePolicy the replicated map merge policy to set - * @return the updated replicated map configuration - * @deprecated since 3.10, please use {@link #setMergePolicyConfig(MergePolicyConfig)} - */ - public ReplicatedMapConfig setMergePolicy(String mergePolicy) { - this.mergePolicyConfig.setPolicy(mergePolicy); - return this; - } - /** * Gets the {@link MergePolicyConfig} for this replicated map. * diff --git a/hazelcast/src/main/java/com/hazelcast/config/ReplicatedMapConfigReadOnly.java b/hazelcast/src/main/java/com/hazelcast/config/ReplicatedMapConfigReadOnly.java index 22d38d0bad60c..6383eb41b3d0b 100644 --- a/hazelcast/src/main/java/com/hazelcast/config/ReplicatedMapConfigReadOnly.java +++ b/hazelcast/src/main/java/com/hazelcast/config/ReplicatedMapConfigReadOnly.java @@ -73,11 +73,6 @@ public ReplicatedMapConfig setQuorumName(String quorumName) { throw throwReadOnly(); } - @Override - public ReplicatedMapConfig setMergePolicy(String mergePolicy) { - throw throwReadOnly(); - } - @Override public ReplicatedMapConfig setMergePolicyConfig(MergePolicyConfig mergePolicyConfig) { throw throwReadOnly(); diff --git a/hazelcast/src/main/java/com/hazelcast/core/IdGenerator.java b/hazelcast/src/main/java/com/hazelcast/core/IdGenerator.java index 2e4ccaa8ce278..90232872cf690 100644 --- a/hazelcast/src/main/java/com/hazelcast/core/IdGenerator.java +++ b/hazelcast/src/main/java/com/hazelcast/core/IdGenerator.java @@ -46,9 +46,9 @@ *

* Care should be taken if you are using the IdGenerator to provide keys for {@link IMap} or * {@link com.hazelcast.cache.ICache}, as after split brain it could be possible for two unique values to share - * the same key. When a {@link com.hazelcast.map.merge.MapMergePolicy} is applied, one of these unique values will + * the same key. When a {@link com.hazelcast.spi.merge.SplitBrainMergePolicy} is applied, one of these unique values will * have to be discarded. Unless configured, the default merge policy is - * {@link com.hazelcast.map.merge.PutIfAbsentMapMergePolicy}. + * {@link com.hazelcast.spi.merge.PutIfAbsentMergePolicy}. *

* It is NOT RECOMMENDED to use {@link IdGenerator} to provide keys that are also used as unique identifiers in * underlying persistent storage, for example in an {@link IMap} that writes to a relational database using {@link diff --git a/hazelcast/src/main/java/com/hazelcast/internal/config/ConfigValidator.java b/hazelcast/src/main/java/com/hazelcast/internal/config/ConfigValidator.java index b3a7ae34f68a9..57de55f329d74 100644 --- a/hazelcast/src/main/java/com/hazelcast/internal/config/ConfigValidator.java +++ b/hazelcast/src/main/java/com/hazelcast/internal/config/ConfigValidator.java @@ -17,7 +17,6 @@ package com.hazelcast.internal.config; import com.hazelcast.cache.ICache; -import com.hazelcast.cache.impl.merge.policy.CacheMergePolicyProvider; import com.hazelcast.config.AbstractBasicConfig; import com.hazelcast.config.CacheConfig; import com.hazelcast.config.CacheSimpleConfig; @@ -50,7 +49,6 @@ import com.hazelcast.internal.eviction.EvictionPolicyComparator; import com.hazelcast.logging.ILogger; import com.hazelcast.logging.Logger; -import com.hazelcast.map.merge.MergePolicyProvider; import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; import com.hazelcast.spi.merge.SplitBrainMergeTypeProvider; import com.hazelcast.spi.properties.HazelcastProperties; @@ -77,10 +75,8 @@ import static com.hazelcast.instance.ProtocolType.MEMBER; import static com.hazelcast.instance.ProtocolType.REST; import static com.hazelcast.instance.ProtocolType.WAN; -import static com.hazelcast.internal.config.MergePolicyValidator.checkCacheMergePolicy; import static com.hazelcast.internal.config.MergePolicyValidator.checkMapMergePolicy; import static com.hazelcast.internal.config.MergePolicyValidator.checkMergePolicy; -import static com.hazelcast.internal.config.MergePolicyValidator.checkReplicatedMapMergePolicy; import static com.hazelcast.spi.properties.GroupProperty.HOT_RESTART_FREE_NATIVE_MEMORY_PERCENTAGE; import static com.hazelcast.spi.properties.GroupProperty.HTTP_HEALTHCHECK_ENABLED; import static com.hazelcast.spi.properties.GroupProperty.MEMCACHE_ENABLED; @@ -113,11 +109,11 @@ private ConfigValidator() { /** * Validates the given {@link MapConfig}. * - * @param mapConfig the {@link MapConfig} - * @param mergePolicyProvider the {@link MergePolicyProvider} to resolve merge policy classes + * @param mapConfig the {@link MapConfig} */ public static void checkMapConfig(MapConfig mapConfig, NativeMemoryConfig nativeMemoryConfig, - MergePolicyProvider mergePolicyProvider, HazelcastProperties properties) { + SplitBrainMergePolicyProvider mergePolicyProvider, + HazelcastProperties properties) { checkNotNativeWhenOpenSource(mapConfig.getInMemoryFormat()); boolean enterprise = getBuildInfo().isEnterprise(); @@ -391,9 +387,10 @@ public static void checkEvictionConfig(EvictionPolicy evictionPolicy, String com * Validates the given {@link CacheSimpleConfig}. * * @param cacheSimpleConfig the {@link CacheSimpleConfig} to check - * @param mergePolicyProvider the {@link CacheMergePolicyProvider} to resolve merge policy classes + * @param mergePolicyProvider the {@link SplitBrainMergePolicyProvider} to resolve merge policy classes */ - public static void checkCacheConfig(CacheSimpleConfig cacheSimpleConfig, CacheMergePolicyProvider mergePolicyProvider) { + public static void checkCacheConfig(CacheSimpleConfig cacheSimpleConfig, + SplitBrainMergePolicyProvider mergePolicyProvider) { checkCacheConfig(cacheSimpleConfig.getInMemoryFormat(), cacheSimpleConfig.getEvictionConfig(), cacheSimpleConfig.getMergePolicy(), cacheSimpleConfig, mergePolicyProvider); } @@ -402,9 +399,9 @@ public static void checkCacheConfig(CacheSimpleConfig cacheSimpleConfig, CacheMe * Validates the given {@link CacheConfig}. * * @param cacheConfig the {@link CacheConfig} to check - * @param mergePolicyProvider the {@link CacheMergePolicyProvider} to resolve merge policy classes + * @param mergePolicyProvider the {@link SplitBrainMergePolicyProvider} to resolve merge policy classes */ - public static void checkCacheConfig(CacheConfig cacheConfig, CacheMergePolicyProvider mergePolicyProvider) { + public static void checkCacheConfig(CacheConfig cacheConfig, SplitBrainMergePolicyProvider mergePolicyProvider) { checkCacheConfig(cacheConfig.getInMemoryFormat(), cacheConfig.getEvictionConfig(), cacheConfig.getMergePolicy(), cacheConfig, mergePolicyProvider); } @@ -416,14 +413,15 @@ public static void checkCacheConfig(CacheConfig cacheConfig, CacheMergePolicyPro * @param evictionConfig the {@link EvictionConfig} of the cache * @param mergePolicyClassname the configured merge policy of the cache * @param mergeTypeProvider the {@link SplitBrainMergeTypeProvider} of the cache - * @param mergePolicyProvider the {@link CacheMergePolicyProvider} to resolve merge policy classes + * @param mergePolicyProvider the {@link SplitBrainMergePolicyProvider} to resolve merge policy classes */ - public static void checkCacheConfig(InMemoryFormat inMemoryFormat, EvictionConfig evictionConfig, String mergePolicyClassname, + public static void checkCacheConfig(InMemoryFormat inMemoryFormat, EvictionConfig evictionConfig, + String mergePolicyClassname, SplitBrainMergeTypeProvider mergeTypeProvider, - CacheMergePolicyProvider mergePolicyProvider) { + SplitBrainMergePolicyProvider mergePolicyProvider) { checkNotNativeWhenOpenSource(inMemoryFormat); checkEvictionConfig(inMemoryFormat, evictionConfig); - checkCacheMergePolicy(mergePolicyClassname, mergeTypeProvider, mergePolicyProvider); + checkMergePolicy(mergeTypeProvider, mergePolicyProvider, mergePolicyClassname); } /** @@ -455,8 +453,8 @@ static void checkEvictionConfig(InMemoryFormat inMemoryFormat, EvictionConfig ev * to resolve merge policy classes */ public static void checkReplicatedMapConfig(ReplicatedMapConfig replicatedMapConfig, - com.hazelcast.replicatedmap.merge.MergePolicyProvider mergePolicyProvider) { - checkReplicatedMapMergePolicy(replicatedMapConfig, mergePolicyProvider); + SplitBrainMergePolicyProvider mergePolicyProvider) { + checkMergePolicy(replicatedMapConfig, mergePolicyProvider, replicatedMapConfig.getMergePolicyConfig().getPolicy()); } /** diff --git a/hazelcast/src/main/java/com/hazelcast/internal/config/MergePolicyValidator.java b/hazelcast/src/main/java/com/hazelcast/internal/config/MergePolicyValidator.java index 2e6fa8bf76c7a..1f262938f2ab0 100644 --- a/hazelcast/src/main/java/com/hazelcast/internal/config/MergePolicyValidator.java +++ b/hazelcast/src/main/java/com/hazelcast/internal/config/MergePolicyValidator.java @@ -16,14 +16,10 @@ package com.hazelcast.internal.config; -import com.hazelcast.cache.ICache; -import com.hazelcast.cache.impl.merge.policy.CacheMergePolicyProvider; import com.hazelcast.config.InMemoryFormat; import com.hazelcast.config.InvalidConfigurationException; import com.hazelcast.config.MapConfig; -import com.hazelcast.config.ReplicatedMapConfig; import com.hazelcast.logging.ILogger; -import com.hazelcast.map.merge.MergePolicyProvider; import com.hazelcast.spi.merge.MergingExpirationTime; import com.hazelcast.spi.merge.MergingLastStoredTime; import com.hazelcast.spi.merge.MergingValue; @@ -61,7 +57,7 @@ private MergePolicyValidator() { * the supplied {@code mergePolicy}, {@code false} otherwise */ public static boolean checkMergePolicySupportsInMemoryFormat(String name, Object mergePolicy, InMemoryFormat inMemoryFormat, - boolean failFast, ILogger logger) { + boolean failFast, ILogger logger) { if (inMemoryFormat != NATIVE) { return true; } @@ -82,79 +78,20 @@ private static String createSplitRecoveryWarningMsg(String name, String mergePol return format(messageTemplate, name, mergePolicy, NATIVE, SplitBrainMergePolicy.class.getName(), V3_10); } - /** - * Checks the merge policy configuration in the context of an {@link ICache}. - * - * @param mergePolicyClassname the configured merge policy of the cache - * @param mergeTypeProvider the {@link SplitBrainMergeTypeProvider} of the cache - * @param mergePolicyProvider the {@link CacheMergePolicyProvider} to resolve merge policy classes - */ - static void checkCacheMergePolicy(String mergePolicyClassname, SplitBrainMergeTypeProvider mergeTypeProvider, - CacheMergePolicyProvider mergePolicyProvider) { - if (mergePolicyProvider == null) { - return; - } - Object mergePolicyInstance = getMergePolicyInstance(mergePolicyProvider, mergePolicyClassname); - checkMergePolicy(mergeTypeProvider, mergePolicyInstance); - } - - private static Object getMergePolicyInstance(CacheMergePolicyProvider mergePolicyProvider, String mergePolicyClassname) { - try { - return mergePolicyProvider.getMergePolicy(mergePolicyClassname); - } catch (InvalidConfigurationException e) { - throw new InvalidConfigurationException("Merge policy must be an instance of SplitBrainMergePolicy" - + " or CacheMergePolicy, but was " + mergePolicyClassname, e.getCause()); - } - } - - /** - * Checks the merge policy configuration of the given {@link ReplicatedMapConfig}. - * - * @param replicatedMapConfig the {@link ReplicatedMapConfig} to check - * @param mergePolicyProvider the {@link com.hazelcast.replicatedmap.merge.MergePolicyProvider} - * to resolve merge policy classes - */ - static void checkReplicatedMapMergePolicy(ReplicatedMapConfig replicatedMapConfig, - com.hazelcast.replicatedmap.merge.MergePolicyProvider mergePolicyProvider) { - String mergePolicyClassName = replicatedMapConfig.getMergePolicyConfig().getPolicy(); - Object mergePolicyInstance = getMergePolicyInstance(mergePolicyProvider, mergePolicyClassName); - checkMergePolicy(replicatedMapConfig, mergePolicyInstance); - } - - private static Object getMergePolicyInstance(com.hazelcast.replicatedmap.merge.MergePolicyProvider mergePolicyProvider, - String mergePolicyClassName) { - try { - return mergePolicyProvider.getMergePolicy(mergePolicyClassName); - } catch (InvalidConfigurationException e) { - throw new InvalidConfigurationException("Merge policy must be an instance of SplitBrainMergePolicy" - + " or ReplicatedMapMergePolicy, but was " + mergePolicyClassName, e.getCause()); - } - } - /** * Checks the merge policy configuration of the given {@link MapConfig}. * - * @param mapConfig the {@link MapConfig} - * @param mergePolicyProvider the {@link MergePolicyProvider} to resolve merge policy classes + * @param mapConfig the {@link MapConfig} */ - static void checkMapMergePolicy(MapConfig mapConfig, MergePolicyProvider mergePolicyProvider) { + static void checkMapMergePolicy(MapConfig mapConfig, SplitBrainMergePolicyProvider mergePolicyProvider) { String mergePolicyClassName = mapConfig.getMergePolicyConfig().getPolicy(); - Object mergePolicyInstance = getMergePolicyInstance(mergePolicyProvider, mergePolicyClassName); - List requiredMergeTypes = checkMergePolicy(mapConfig, mergePolicyInstance); + SplitBrainMergePolicy mergePolicyInstance = mergePolicyProvider.getMergePolicy(mergePolicyClassName); + List requiredMergeTypes = checkSplitBrainMergePolicy(mapConfig, mergePolicyInstance); if (!mapConfig.isStatisticsEnabled() && requiredMergeTypes != null) { checkMapMergePolicyWhenStatisticsAreDisabled(mergePolicyClassName, requiredMergeTypes); } } - private static Object getMergePolicyInstance(MergePolicyProvider mergePolicyProvider, String mergePolicyClassName) { - try { - return mergePolicyProvider.getMergePolicy(mergePolicyClassName); - } catch (InvalidConfigurationException e) { - throw new InvalidConfigurationException("Merge policy must be an instance of SplitBrainMergePolicy" - + " or MapMergePolicy, but was " + mergePolicyClassName, e.getCause()); - } - } - /** * Checks if the configured merge policy requires merge types, which are just available if map statistics are enabled. * @@ -181,8 +118,14 @@ private static void checkMapMergePolicyWhenStatisticsAreDisabled(String mergePol * @throws InvalidConfigurationException if the given merge policy is no {@link SplitBrainMergePolicy} */ static void checkMergePolicy(SplitBrainMergeTypeProvider mergeTypeProvider, - SplitBrainMergePolicyProvider mergePolicyProvider, String mergePolicyClassName) { - SplitBrainMergePolicy mergePolicy = getMergePolicyInstance(mergePolicyProvider, mergePolicyClassName); + SplitBrainMergePolicyProvider mergePolicyProvider, + String mergePolicyClassName) { + if (mergePolicyProvider == null) { + return; + } + + SplitBrainMergePolicy mergePolicy = getMergePolicyInstance(mergePolicyProvider, + mergePolicyClassName); checkSplitBrainMergePolicy(mergeTypeProvider, mergePolicy); } @@ -196,20 +139,6 @@ private static SplitBrainMergePolicy getMergePolicyInstance(SplitBrainMergePolic } } - /** - * Checks if a {@link SplitBrainMergeTypeProvider} provides all required types of a given merge policy instance. - * - * @param mergeTypeProvider the {@link SplitBrainMergeTypeProvider} to retrieve the provided merge types - * @param mergePolicyInstance the merge policy instance - * @return a list of the required merge types if the merge policy is a {@link SplitBrainMergePolicy}, {@code null} otherwise - */ - private static List checkMergePolicy(SplitBrainMergeTypeProvider mergeTypeProvider, Object mergePolicyInstance) { - if (mergePolicyInstance instanceof SplitBrainMergePolicy) { - return checkSplitBrainMergePolicy(mergeTypeProvider, (SplitBrainMergePolicy) mergePolicyInstance); - } - return null; - } - /** * Checks if a {@link SplitBrainMergeTypeProvider} provides all required types of a given {@link SplitBrainMergePolicy}. * diff --git a/hazelcast/src/main/java/com/hazelcast/internal/management/dto/MapConfigDTO.java b/hazelcast/src/main/java/com/hazelcast/internal/management/dto/MapConfigDTO.java index 34aa92db91188..ea6cb94080311 100644 --- a/hazelcast/src/main/java/com/hazelcast/internal/management/dto/MapConfigDTO.java +++ b/hazelcast/src/main/java/com/hazelcast/internal/management/dto/MapConfigDTO.java @@ -20,9 +20,9 @@ import com.hazelcast.config.InMemoryFormat; import com.hazelcast.config.MapConfig; import com.hazelcast.config.MaxSizeConfig; +import com.hazelcast.internal.json.JsonObject; import com.hazelcast.internal.management.JsonSerializable; import com.hazelcast.internal.management.ManagementDataSerializerHook; -import com.hazelcast.internal.json.JsonObject; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; @@ -62,7 +62,6 @@ public JsonObject toJson() { root.add("maxSizePolicy", config.getMaxSizeConfig().getMaxSizePolicy().toString()); root.add("readBackupData", config.isReadBackupData()); root.add("evictionPolicy", config.getEvictionPolicy().name()); - root.add("mergePolicy", config.getMergePolicy()); return root; } @@ -81,7 +80,6 @@ public void fromJson(JsonObject json) { .setMaxSizePolicy(MaxSizeConfig.MaxSizePolicy.valueOf(getString(json, "maxSizePolicy")))); config.setReadBackupData(getBoolean(json, "readBackupData")); config.setEvictionPolicy(EvictionPolicy.valueOf(getString(json, "evictionPolicy"))); - config.setMergePolicy(getString(json, "mergePolicy")); } @Override @@ -101,7 +99,6 @@ public void readData(ObjectDataInput in) throws IOException { .setMaxSizePolicy(MaxSizeConfig.MaxSizePolicy.valueOf(in.readUTF()))); config.setReadBackupData(in.readBoolean()); config.setEvictionPolicy(EvictionPolicy.valueOf(in.readUTF())); - config.setMergePolicy(in.readUTF()); } @Override @@ -118,7 +115,6 @@ public void writeData(ObjectDataOutput out) throws IOException { out.writeUTF(config.getMaxSizeConfig().getMaxSizePolicy().toString()); out.writeBoolean(config.isReadBackupData()); out.writeUTF(config.getEvictionPolicy().name()); - out.writeUTF(config.getMergePolicy()); } public MapConfig getMapConfig() { diff --git a/hazelcast/src/main/java/com/hazelcast/map/IMap.java b/hazelcast/src/main/java/com/hazelcast/map/IMap.java index c7277aa204524..6a2bbaf8cad1c 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/IMap.java +++ b/hazelcast/src/main/java/com/hazelcast/map/IMap.java @@ -135,7 +135,7 @@ * will continue to use the primary or back-up version. *

* When the split heals, Hazelcast by default, performs a - * {@link com.hazelcast.map.merge.PutIfAbsentMapMergePolicy}. + * {@link com.hazelcast.spi.merge.PutIfAbsentMergePolicy}. * Users can also decide to * * specify their own map merge policies, these policies when used in diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/EntryViews.java b/hazelcast/src/main/java/com/hazelcast/map/impl/EntryViews.java index 24a43d81dcdee..534392573ec86 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/EntryViews.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/EntryViews.java @@ -18,8 +18,6 @@ import com.hazelcast.core.EntryView; import com.hazelcast.map.impl.record.Record; -import com.hazelcast.map.merge.MapMergePolicy; -import com.hazelcast.spi.serialization.SerializationService; /** * A class providing static factory methods that create various entry view objects. @@ -29,19 +27,6 @@ public final class EntryViews { private EntryViews() { } - /** - * Creates a null entry view that has only key and no value. - * - * @param key the key object which will be wrapped in {@link com.hazelcast.core.EntryView}. - * @param the type of key. - * @param the type of value. - * @return returns null entry view. - */ - public static EntryView createLazyNullEntryView(K key, - SerializationService serializationService) { - return new LazyNullEntryView<>(key, serializationService); - } - public static EntryView createSimpleEntryView() { return new SimpleEntryView<>(); } @@ -59,36 +44,4 @@ public static EntryView createSimpleEntryView(K key, V value, Recor .withExpirationTime(record.getExpirationTime()) .withLastStoredTime(record.getLastStoredTime()); } - - public static EntryView createLazyEntryView(K key, V value, Record record, - SerializationService serializationService, - MapMergePolicy mergePolicy) { - return new LazyEntryView<>(key, value, serializationService, mergePolicy) - .setCost(record.getCost()) - .setVersion(record.getVersion()) - .setHits(record.getHits()) - .setLastAccessTime(record.getLastAccessTime()) - .setLastUpdateTime(record.getLastUpdateTime()) - .setTtl(record.getTtl()) - .setMaxIdle(record.getMaxIdle()) - .setCreationTime(record.getCreationTime()) - .setExpirationTime(record.getExpirationTime()) - .setLastStoredTime(record.getLastStoredTime()); - } - - public static EntryView toLazyEntryView(EntryView entryView, - SerializationService serializationService, - MapMergePolicy mergePolicy) { - return new LazyEntryView<>(entryView.getKey(), entryView.getValue(), serializationService, mergePolicy) - .setCost(entryView.getCost()) - .setVersion(entryView.getVersion()) - .setLastAccessTime(entryView.getLastAccessTime()) - .setLastUpdateTime(entryView.getLastUpdateTime()) - .setTtl(entryView.getTtl()) - .setMaxIdle(entryView.getMaxIdle()) - .setCreationTime(entryView.getCreationTime()) - .setHits(entryView.getHits()) - .setExpirationTime(entryView.getExpirationTime()) - .setLastStoredTime(entryView.getLastStoredTime()); - } } diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/LazyEntryView.java b/hazelcast/src/main/java/com/hazelcast/map/impl/LazyEntryView.java deleted file mode 100644 index 85bbd5bc9414a..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/LazyEntryView.java +++ /dev/null @@ -1,201 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.impl; - -import com.hazelcast.core.EntryView; -import com.hazelcast.map.merge.HigherHitsMapMergePolicy; -import com.hazelcast.map.merge.LatestUpdateMapMergePolicy; -import com.hazelcast.map.merge.MapMergePolicy; -import com.hazelcast.map.merge.PassThroughMergePolicy; -import com.hazelcast.map.merge.PutIfAbsentMapMergePolicy; -import com.hazelcast.spi.serialization.SerializationService; - -/** - * LazyEntryView is an implementation of {@link com.hazelcast.core.EntryView} and also it is writable. - * - * @param the type of key. - * @param the type of value. - */ - -class LazyEntryView implements EntryView { - - private K key; - private V value; - private long cost; - private long creationTime; - private long expirationTime; - private long hits; - private long lastAccessTime; - private long lastStoredTime; - private long lastUpdateTime; - private long version; - private long ttl; - private Long maxIdle; - - private SerializationService serializationService; - private MapMergePolicy mergePolicy; - - LazyEntryView() { - } - - LazyEntryView(K key, V value, SerializationService serializationService, MapMergePolicy mergePolicy) { - this.value = value; - this.key = key; - this.serializationService = serializationService; - this.mergePolicy = mergePolicy; - } - - @Override - public K getKey() { - key = serializationService.toObject(key); - return key; - } - - public LazyEntryView setKey(K key) { - this.key = key; - return this; - } - - @Override - public V getValue() { - if (returnRawData(mergePolicy)) { - return value; - } - value = serializationService.toObject(value); - return value; - } - - private boolean returnRawData(MapMergePolicy mergePolicy) { - return mergePolicy instanceof PutIfAbsentMapMergePolicy - || mergePolicy instanceof PassThroughMergePolicy - || mergePolicy instanceof HigherHitsMapMergePolicy - || mergePolicy instanceof LatestUpdateMapMergePolicy; - } - - public LazyEntryView setValue(V value) { - this.value = value; - return this; - } - - @Override - public long getCost() { - return cost; - } - - public LazyEntryView setCost(long cost) { - this.cost = cost; - return this; - } - - @Override - public long getCreationTime() { - return creationTime; - } - - public LazyEntryView setCreationTime(long creationTime) { - this.creationTime = creationTime; - return this; - } - - @Override - public long getExpirationTime() { - return expirationTime; - } - - public LazyEntryView setExpirationTime(long expirationTime) { - this.expirationTime = expirationTime; - return this; - } - - @Override - public long getHits() { - return hits; - } - - public LazyEntryView setHits(long hits) { - this.hits = hits; - return this; - } - - @Override - public long getLastAccessTime() { - return lastAccessTime; - } - - public LazyEntryView setLastAccessTime(long lastAccessTime) { - this.lastAccessTime = lastAccessTime; - return this; - } - - @Override - public long getLastStoredTime() { - return lastStoredTime; - } - - public LazyEntryView setLastStoredTime(long lastStoredTime) { - this.lastStoredTime = lastStoredTime; - return this; - } - - @Override - public long getLastUpdateTime() { - return lastUpdateTime; - } - - public LazyEntryView setLastUpdateTime(long lastUpdateTime) { - this.lastUpdateTime = lastUpdateTime; - return this; - } - - @Override - public long getVersion() { - return version; - } - - public LazyEntryView setVersion(long version) { - this.version = version; - return this; - } - - public long getEvictionCriteriaNumber() { - return 0; - } - - public LazyEntryView setEvictionCriteriaNumber(long evictionCriteriaNumber) { - return this; - } - - @Override - public long getTtl() { - return ttl; - } - - public LazyEntryView setTtl(long ttl) { - this.ttl = ttl; - return this; - } - - @Override - public Long getMaxIdle() { - return maxIdle; - } - - public LazyEntryView setMaxIdle(Long maxIdle) { - this.maxIdle = maxIdle; - return this; - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/MapContainer.java b/hazelcast/src/main/java/com/hazelcast/map/impl/MapContainer.java index 989c57bbf1b1e..9d0543d08aaf0 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/MapContainer.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/MapContainer.java @@ -50,6 +50,7 @@ import com.hazelcast.query.impl.getters.Extractors; import com.hazelcast.spi.NodeEngine; import com.hazelcast.spi.ObjectNamespace; +import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.partition.IPartitionService; import com.hazelcast.spi.serialization.SerializationService; import com.hazelcast.util.ConstructorFunction; @@ -100,7 +101,7 @@ public class MapContainer { */ protected final AtomicInteger invalidationListenerCount = new AtomicInteger(); - protected Object wanMergePolicy; + protected SplitBrainMergePolicy wanMergePolicy; protected WanReplicationPublisher wanReplicationPublisher; protected volatile Evictor evictor; @@ -251,7 +252,7 @@ && hasPublisherWithMerkleTreeSync(config, wanReplicationRefName)) { WanReplicationService wanReplicationService = nodeEngine.getWanReplicationService(); wanReplicationPublisher = wanReplicationService.getWanReplicationPublisher(wanReplicationRefName); - wanMergePolicy = mapServiceContext.getMergePolicyProvider().getMergePolicy(wanReplicationRef.getMergePolicy()); + wanMergePolicy = nodeEngine.getSplitBrainMergePolicyProvider().getMergePolicy(wanReplicationRef.getMergePolicy()); WanReplicationConfig wanReplicationConfig = config.getWanReplicationConfig(wanReplicationRefName); if (wanReplicationConfig != null) { @@ -315,7 +316,7 @@ public WanReplicationPublisher getWanReplicationPublisher() { return wanReplicationPublisher; } - public Object getWanMergePolicy() { + public SplitBrainMergePolicy getWanMergePolicy() { return wanMergePolicy; } diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/MapDataSerializerHook.java b/hazelcast/src/main/java/com/hazelcast/map/impl/MapDataSerializerHook.java index b3b0e80b2826f..9626b7fa364bb 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/MapDataSerializerHook.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/MapDataSerializerHook.java @@ -63,7 +63,6 @@ import com.hazelcast.map.impl.operation.IsPartitionLoadedOperationFactory; import com.hazelcast.map.impl.operation.KeyLoadStatusOperation; import com.hazelcast.map.impl.operation.KeyLoadStatusOperationFactory; -import com.hazelcast.map.impl.operation.LegacyMergeOperation; import com.hazelcast.map.impl.operation.LoadAllOperation; import com.hazelcast.map.impl.operation.LoadMapOperation; import com.hazelcast.map.impl.operation.MapFetchEntriesOperation; @@ -149,10 +148,6 @@ import com.hazelcast.map.impl.tx.TxnUnlockBackupOperation; import com.hazelcast.map.impl.tx.TxnUnlockOperation; import com.hazelcast.map.impl.tx.VersionedValue; -import com.hazelcast.map.merge.HigherHitsMapMergePolicy; -import com.hazelcast.map.merge.LatestUpdateMapMergePolicy; -import com.hazelcast.map.merge.PassThroughMergePolicy; -import com.hazelcast.map.merge.PutIfAbsentMapMergePolicy; import com.hazelcast.nio.serialization.DataSerializableFactory; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; import com.hazelcast.query.impl.IndexInfo; @@ -201,124 +196,115 @@ public final class MapDataSerializerHook implements DataSerializerHook { public static final int EVICT_ALL_BACKUP = 32; public static final int GET_ALL = 33; public static final int IS_EMPTY = 34; - public static final int LEGACY_MERGE = 35; - public static final int NEAR_CACHE_SINGLE_INVALIDATION = 36; - public static final int NEAR_CACHE_BATCH_INVALIDATION = 37; - public static final int IS_PARTITION_LOADED = 38; - public static final int PARTITION_WIDE_ENTRY = 39; - public static final int PARTITION_WIDE_ENTRY_BACKUP = 40; - public static final int PARTITION_WIDE_PREDICATE_ENTRY = 41; - public static final int PARTITION_WIDE_PREDICATE_ENTRY_BACKUP = 42; - public static final int ADD_INDEX = 43; - public static final int AWAIT_MAP_FLUSH = 44; - public static final int CONTAINS_VALUE = 45; - public static final int GET_ENTRY_VIEW = 46; - public static final int FETCH_ENTRIES = 47; - public static final int FETCH_KEYS = 48; - public static final int FLUSH_BACKUP = 49; - public static final int FLUSH = 50; - public static final int MULTIPLE_ENTRY_BACKUP = 51; - public static final int MULTIPLE_ENTRY = 52; - public static final int MULTIPLE_ENTRY_PREDICATE_BACKUP = 53; - public static final int MULTIPLE_ENTRY_PREDICATE = 54; - public static final int NOTIFY_MAP_FLUSH = 55; - public static final int PUT_IF_ABSENT = 56; - public static final int PUT_FROM_LOAD_ALL = 57; - public static final int PUT_FROM_LOAD_ALL_BACKUP = 58; - public static final int QUERY_PARTITION = 59; - public static final int QUERY_OPERATION = 60; - public static final int PUT_TRANSIENT = 61; - public static final int REPLACE_IF_SAME = 62; - public static final int TRY_PUT = 63; - public static final int TRY_REMOVE = 64; - public static final int TXN_LOCK_AND_GET = 65; - public static final int TXN_DELETE = 66; - public static final int TXN_PREPARE = 67; - public static final int TXN_PREPARE_BACKUP = 68; - public static final int TXN_ROLLBACK = 69; - public static final int TXN_ROLLBACK_BACKUP = 70; - public static final int TXN_SET = 71; - public static final int TXN_UNLOCK = 72; - public static final int TXN_UNLOCK_BACKUP = 73; - public static final int IS_PARTITION_LOADED_FACTORY = 74; - public static final int ADD_INDEX_FACTORY = 75; - public static final int ADD_INTERCEPTOR_FACTORY = 76; - public static final int CLEAR_FACTORY = 77; - public static final int CONTAINS_VALUE_FACTORY = 78; - public static final int EVICT_ALL_FACTORY = 79; - public static final int IS_EMPTY_FACTORY = 80; - public static final int KEY_LOAD_STATUS_FACTORY = 81; - public static final int MAP_FLUSH_FACTORY = 82; - public static final int MAP_GET_ALL_FACTORY = 83; - public static final int LOAD_ALL_FACTORY = 84; - public static final int PARTITION_WIDE_ENTRY_FACTORY = 85; - public static final int PARTITION_WIDE_PREDICATE_ENTRY_FACTORY = 86; - public static final int PUT_ALL_PARTITION_AWARE_FACTORY = 87; - public static final int REMOVE_INTERCEPTOR_FACTORY = 88; - public static final int SIZE_FACTORY = 89; - public static final int MULTIPLE_ENTRY_FACTORY = 90; - public static final int ENTRY_EVENT_FILTER = 91; - public static final int EVENT_LISTENER_FILTER = 92; - public static final int PARTITION_LOST_EVENT_FILTER = 93; - public static final int NEAR_CACHE_CLEAR_INVALIDATION = 94; - public static final int ADD_INTERCEPTOR = 95; - public static final int MAP_REPLICATION = 96; - public static final int POST_JOIN_MAP_OPERATION = 97; - public static final int INDEX_INFO = 98; - public static final int MAP_INDEX_INFO = 99; - public static final int INTERCEPTOR_INFO = 100; - public static final int REMOVE_INTERCEPTOR = 101; - public static final int QUERY_EVENT_FILTER = 102; - public static final int RECORD_INFO = 103; - public static final int RECORD_REPLICATION_INFO = 104; - public static final int HIGHER_HITS_MERGE_POLICY = 105; - public static final int LATEST_UPDATE_MERGE_POLICY = 106; - public static final int PASS_THROUGH_MERGE_POLICY = 107; - public static final int PUT_IF_ABSENT_MERGE_POLICY = 108; - public static final int UUID_FILTER = 109; - public static final int CLEAR_NEAR_CACHE_INVALIDATION = 110; - public static final int MAP_TRANSACTION_LOG_RECORD = 111; - public static final int VERSIONED_VALUE = 112; - public static final int MAP_REPLICATION_STATE_HOLDER = 113; - public static final int WRITE_BEHIND_STATE_HOLDER = 114; - public static final int AGGREGATION_RESULT = 115; - public static final int QUERY = 116; - public static final int TARGET = 117; - public static final int MAP_INVALIDATION_METADATA = 118; - public static final int MAP_INVALIDATION_METADATA_RESPONSE = 119; - public static final int MAP_NEAR_CACHE_STATE_HOLDER = 120; - public static final int MAP_ASSIGN_AND_GET_UUIDS = 121; - public static final int MAP_ASSIGN_AND_GET_UUIDS_FACTORY = 122; - public static final int DESTROY_QUERY_CACHE = 123; - public static final int MADE_PUBLISHABLE = 124; - public static final int MADE_PUBLISHABLE_FACTORY = 125; - public static final int PUBLISHER_CREATE = 126; - public static final int READ_AND_RESET_ACCUMULATOR = 127; - public static final int SET_READ_CURSOR = 128; - public static final int ACCUMULATOR_CONSUMER = 129; - public static final int LAZY_MAP_ENTRY = 131; - public static final int TRIGGER_LOAD_IF_NEEDED = 132; - public static final int IS_KEYLOAD_FINISHED = 133; - public static final int REMOVE_FROM_LOAD_ALL = 134; - public static final int ENTRY_REMOVING_PROCESSOR = 135; - public static final int ENTRY_OFFLOADABLE_SET_UNLOCK = 136; - public static final int LOCK_AWARE_LAZY_MAP_ENTRY = 137; - public static final int FETCH_WITH_QUERY = 138; - public static final int RESULT_SEGMENT = 139; - public static final int EVICT_BATCH_BACKUP = 140; - public static final int EVENT_JOURNAL_SUBSCRIBE_OPERATION = 141; - public static final int EVENT_JOURNAL_READ = 142; - public static final int EVENT_JOURNAL_DESERIALIZING_MAP_EVENT = 143; - public static final int EVENT_JOURNAL_INTERNAL_MAP_EVENT = 144; - public static final int EVENT_JOURNAL_READ_RESULT_SET = 145; - public static final int MERGE_FACTORY = 146; - public static final int MERGE = 147; - public static final int SET_TTL = 148; - public static final int SET_TTL_BACKUP = 149; - public static final int MERKLE_TREE_NODE_ENTRIES = 150; - public static final int ADD_INDEX_BACKUP = 151; - public static final int TXN_SET_BACKUP = 152; - public static final int TXN_DELETE_BACKUP = 153; + public static final int NEAR_CACHE_SINGLE_INVALIDATION = 35; + public static final int NEAR_CACHE_BATCH_INVALIDATION = 36; + public static final int IS_PARTITION_LOADED = 37; + public static final int PARTITION_WIDE_ENTRY = 38; + public static final int PARTITION_WIDE_ENTRY_BACKUP = 39; + public static final int PARTITION_WIDE_PREDICATE_ENTRY = 40; + public static final int PARTITION_WIDE_PREDICATE_ENTRY_BACKUP = 41; + public static final int ADD_INDEX = 42; + public static final int AWAIT_MAP_FLUSH = 43; + public static final int CONTAINS_VALUE = 44; + public static final int GET_ENTRY_VIEW = 45; + public static final int FETCH_ENTRIES = 46; + public static final int FETCH_KEYS = 47; + public static final int FLUSH_BACKUP = 48; + public static final int FLUSH = 49; + public static final int MULTIPLE_ENTRY_BACKUP = 50; + public static final int MULTIPLE_ENTRY = 51; + public static final int MULTIPLE_ENTRY_PREDICATE_BACKUP = 52; + public static final int MULTIPLE_ENTRY_PREDICATE = 53; + public static final int NOTIFY_MAP_FLUSH = 54; + public static final int PUT_IF_ABSENT = 55; + public static final int PUT_FROM_LOAD_ALL = 56; + public static final int PUT_FROM_LOAD_ALL_BACKUP = 57; + public static final int QUERY_PARTITION = 58; + public static final int QUERY_OPERATION = 59; + public static final int PUT_TRANSIENT = 60; + public static final int REPLACE_IF_SAME = 61; + public static final int TRY_PUT = 62; + public static final int TRY_REMOVE = 63; + public static final int TXN_LOCK_AND_GET = 64; + public static final int TXN_DELETE = 65; + public static final int TXN_PREPARE = 66; + public static final int TXN_PREPARE_BACKUP = 67; + public static final int TXN_ROLLBACK = 68; + public static final int TXN_ROLLBACK_BACKUP = 69; + public static final int TXN_SET = 70; + public static final int TXN_UNLOCK = 71; + public static final int TXN_UNLOCK_BACKUP = 72; + public static final int IS_PARTITION_LOADED_FACTORY = 73; + public static final int ADD_INDEX_FACTORY = 74; + public static final int CLEAR_FACTORY = 75; + public static final int CONTAINS_VALUE_FACTORY = 76; + public static final int EVICT_ALL_FACTORY = 77; + public static final int IS_EMPTY_FACTORY = 78; + public static final int KEY_LOAD_STATUS_FACTORY = 79; + public static final int MAP_FLUSH_FACTORY = 80; + public static final int MAP_GET_ALL_FACTORY = 81; + public static final int LOAD_ALL_FACTORY = 82; + public static final int PARTITION_WIDE_ENTRY_FACTORY = 83; + public static final int PARTITION_WIDE_PREDICATE_ENTRY_FACTORY = 84; + public static final int PUT_ALL_PARTITION_AWARE_FACTORY = 85; + public static final int SIZE_FACTORY = 86; + public static final int MULTIPLE_ENTRY_FACTORY = 87; + public static final int ENTRY_EVENT_FILTER = 88; + public static final int EVENT_LISTENER_FILTER = 89; + public static final int PARTITION_LOST_EVENT_FILTER = 90; + public static final int ADD_INTERCEPTOR = 91; + public static final int MAP_REPLICATION = 92; + public static final int POST_JOIN_MAP_OPERATION = 93; + public static final int INDEX_INFO = 94; + public static final int MAP_INDEX_INFO = 95; + public static final int INTERCEPTOR_INFO = 96; + public static final int REMOVE_INTERCEPTOR = 97; + public static final int QUERY_EVENT_FILTER = 98; + public static final int RECORD_INFO = 99; + public static final int RECORD_REPLICATION_INFO = 100; + public static final int UUID_FILTER = 101; + public static final int MAP_TRANSACTION_LOG_RECORD = 102; + public static final int VERSIONED_VALUE = 103; + public static final int MAP_REPLICATION_STATE_HOLDER = 104; + public static final int WRITE_BEHIND_STATE_HOLDER = 105; + public static final int AGGREGATION_RESULT = 106; + public static final int QUERY = 107; + public static final int TARGET = 108; + public static final int MAP_INVALIDATION_METADATA = 109; + public static final int MAP_INVALIDATION_METADATA_RESPONSE = 110; + public static final int MAP_NEAR_CACHE_STATE_HOLDER = 111; + public static final int MAP_ASSIGN_AND_GET_UUIDS = 112; + public static final int MAP_ASSIGN_AND_GET_UUIDS_FACTORY = 113; + public static final int DESTROY_QUERY_CACHE = 114; + public static final int MADE_PUBLISHABLE = 115; + public static final int MADE_PUBLISHABLE_FACTORY = 116; + public static final int PUBLISHER_CREATE = 117; + public static final int READ_AND_RESET_ACCUMULATOR = 118; + public static final int SET_READ_CURSOR = 119; + public static final int ACCUMULATOR_CONSUMER = 120; + public static final int LAZY_MAP_ENTRY = 121; + public static final int TRIGGER_LOAD_IF_NEEDED = 122; + public static final int IS_KEYLOAD_FINISHED = 123; + public static final int REMOVE_FROM_LOAD_ALL = 124; + public static final int ENTRY_REMOVING_PROCESSOR = 125; + public static final int ENTRY_OFFLOADABLE_SET_UNLOCK = 126; + public static final int LOCK_AWARE_LAZY_MAP_ENTRY = 127; + public static final int FETCH_WITH_QUERY = 128; + public static final int RESULT_SEGMENT = 129; + public static final int EVICT_BATCH_BACKUP = 130; + public static final int EVENT_JOURNAL_SUBSCRIBE_OPERATION = 131; + public static final int EVENT_JOURNAL_READ = 132; + public static final int EVENT_JOURNAL_DESERIALIZING_MAP_EVENT = 133; + public static final int EVENT_JOURNAL_INTERNAL_MAP_EVENT = 134; + public static final int EVENT_JOURNAL_READ_RESULT_SET = 135; + public static final int MERGE_FACTORY = 136; + public static final int MERGE = 137; + public static final int SET_TTL = 138; + public static final int SET_TTL_BACKUP = 139; + public static final int MERKLE_TREE_NODE_ENTRIES = 140; + public static final int ADD_INDEX_BACKUP = 141; + public static final int TXN_SET_BACKUP = 142; + public static final int TXN_DELETE_BACKUP = 143; private static final int LEN = TXN_DELETE_BACKUP + 1; @@ -366,7 +352,6 @@ public DataSerializableFactory createFactory() { constructors[EVICT_ALL_BACKUP] = arg -> new EvictAllBackupOperation(); constructors[GET_ALL] = arg -> new GetAllOperation(); constructors[IS_EMPTY] = arg -> new MapIsEmptyOperation(); - constructors[LEGACY_MERGE] = arg -> new LegacyMergeOperation(); constructors[IS_PARTITION_LOADED] = arg -> new IsPartitionLoadedOperation(); constructors[PARTITION_WIDE_ENTRY] = arg -> new PartitionWideEntryOperation(); constructors[PARTITION_WIDE_ENTRY_BACKUP] = arg -> new PartitionWideEntryBackupOperation(); @@ -433,10 +418,6 @@ public DataSerializableFactory createFactory() { constructors[QUERY_EVENT_FILTER] = arg -> new QueryEventFilter(); constructors[RECORD_INFO] = arg -> new RecordInfo(); constructors[RECORD_REPLICATION_INFO] = arg -> new RecordReplicationInfo(); - constructors[HIGHER_HITS_MERGE_POLICY] = arg -> new HigherHitsMapMergePolicy(); - constructors[LATEST_UPDATE_MERGE_POLICY] = arg -> new LatestUpdateMapMergePolicy(); - constructors[PASS_THROUGH_MERGE_POLICY] = arg -> new PassThroughMergePolicy(); - constructors[PUT_IF_ABSENT_MERGE_POLICY] = arg -> new PutIfAbsentMapMergePolicy(); constructors[UUID_FILTER] = arg -> new UuidFilter(); constructors[MAP_TRANSACTION_LOG_RECORD] = arg -> new MapTransactionLogRecord(); constructors[VERSIONED_VALUE] = arg -> new VersionedValue(); diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/MapMergeRunnable.java b/hazelcast/src/main/java/com/hazelcast/map/impl/MapMergeRunnable.java index 1508649e92b55..c4ee0f4b5d76d 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/MapMergeRunnable.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/MapMergeRunnable.java @@ -19,15 +19,12 @@ import com.hazelcast.config.InMemoryFormat; import com.hazelcast.config.MapConfig; import com.hazelcast.config.MergePolicyConfig; -import com.hazelcast.core.EntryView; import com.hazelcast.map.impl.operation.MapOperationProvider; import com.hazelcast.map.impl.record.Record; import com.hazelcast.map.impl.recordstore.RecordStore; -import com.hazelcast.map.merge.MapMergePolicy; import com.hazelcast.nio.serialization.Data; -import com.hazelcast.spi.impl.operationservice.Operation; -import com.hazelcast.spi.impl.operationservice.OperationFactory; import com.hazelcast.spi.impl.merge.AbstractMergeRunnable; +import com.hazelcast.spi.impl.operationservice.OperationFactory; import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergeTypes.MapMergeTypes; import com.hazelcast.util.Clock; @@ -37,7 +34,6 @@ import java.util.List; import java.util.function.BiConsumer; -import static com.hazelcast.map.impl.EntryViews.createSimpleEntryView; import static com.hazelcast.spi.impl.merge.MergingValueFactory.createMergingEntry; class MapMergeRunnable extends AbstractMergeRunnable { @@ -69,28 +65,6 @@ protected void mergeStore(RecordStore store, BiConsumer } } - @Override - protected void mergeStoreLegacy(RecordStore store, BiConsumer consumer) { - long now = Clock.currentTimeMillis(); - int partitionId = store.getPartitionId(); - String name = store.getName(); - MapOperationProvider operationProvider = mapServiceContext.getMapOperationProvider(name); - MapMergePolicy mergePolicy = ((MapMergePolicy) getMergePolicy(name)); - - //noinspection unchecked - Iterator iterator = store.iterator(now, false); - while (iterator.hasNext()) { - Record record = iterator.next(); - Data key = record.getKey(); - Data value = toData(record.getValue()); - EntryView entryView = createSimpleEntryView(key, value, record); - - Operation operation = operationProvider.createLegacyMergeOperation(name, entryView, mergePolicy, false); - - consumer.accept(partitionId, operation); - } - } - @Override protected int getBatchSize(String dataStructureName) { MapConfig mapConfig = getMapConfig(dataStructureName); @@ -105,8 +79,10 @@ protected InMemoryFormat getInMemoryFormat(String dataStructureName) { } @Override - protected Object getMergePolicy(String dataStructureName) { - return mapServiceContext.getMergePolicy(dataStructureName); + protected SplitBrainMergePolicy getMergePolicy(String dataStructureName) { + MapConfig mapConfig = getMapConfig(dataStructureName); + MergePolicyConfig mergePolicyConfig = mapConfig.getMergePolicyConfig(); + return mergePolicyProvider.getMergePolicy(mergePolicyConfig.getPolicy()); } @Override diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/MapRemoteService.java b/hazelcast/src/main/java/com/hazelcast/map/impl/MapRemoteService.java index 89dcb7a1a819d..741d7af2251ba 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/MapRemoteService.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/MapRemoteService.java @@ -21,9 +21,9 @@ import com.hazelcast.core.DistributedObject; import com.hazelcast.map.impl.proxy.MapProxyImpl; import com.hazelcast.map.impl.proxy.NearCachedMapProxyImpl; -import com.hazelcast.map.merge.MergePolicyProvider; import com.hazelcast.spi.NodeEngine; import com.hazelcast.spi.RemoteService; +import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; import static com.hazelcast.config.NearCacheConfigAccessor.initDefaultMaxSizeForOnHeapMaps; import static com.hazelcast.internal.config.ConfigValidator.checkMapConfig; @@ -49,7 +49,7 @@ class MapRemoteService implements RemoteService { public DistributedObject createDistributedObject(String name) { Config config = nodeEngine.getConfig(); MapConfig mapConfig = config.findMapConfig(name); - MergePolicyProvider mergePolicyProvider = mapServiceContext.getMergePolicyProvider(); + SplitBrainMergePolicyProvider mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); checkMapConfig(mapConfig, config.getNativeMemoryConfig(), mergePolicyProvider, mapServiceContext.getNodeEngine().getProperties()); diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/MapReplicationSupportingService.java b/hazelcast/src/main/java/com/hazelcast/map/impl/MapReplicationSupportingService.java index d12bfce31e093..ce2c57fa2fee5 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/MapReplicationSupportingService.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/MapReplicationSupportingService.java @@ -16,20 +16,18 @@ package com.hazelcast.map.impl; -import com.hazelcast.core.EntryView; import com.hazelcast.map.impl.operation.MapOperation; import com.hazelcast.map.impl.operation.MapOperationProvider; import com.hazelcast.map.impl.wan.MapReplicationRemove; import com.hazelcast.map.impl.wan.MapReplicationUpdate; -import com.hazelcast.map.merge.MapMergePolicy; import com.hazelcast.nio.serialization.Data; import com.hazelcast.spi.NodeEngine; import com.hazelcast.spi.ReplicationSupportingService; import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergeTypes.MapMergeTypes; import com.hazelcast.spi.serialization.SerializationService; -import com.hazelcast.wan.WanReplicationEvent; import com.hazelcast.wan.DistributedServiceWanEventCounters; +import com.hazelcast.wan.WanReplicationEvent; import java.util.concurrent.Future; @@ -46,7 +44,7 @@ class MapReplicationSupportingService implements ReplicationSupportingService { this.mapServiceContext = mapServiceContext; this.nodeEngine = mapServiceContext.getNodeEngine(); this.wanEventTypeCounters = nodeEngine.getWanReplicationService() - .getReceivedEventCounters(MapService.SERVICE_NAME); + .getReceivedEventCounters(MapService.SERVICE_NAME); } @Override @@ -77,21 +75,16 @@ private void handleRemove(MapReplicationRemove replicationRemove) { } private void handleUpdate(MapReplicationUpdate replicationUpdate) { - Object mergePolicy = replicationUpdate.getMergePolicy(); + SplitBrainMergePolicy mergePolicy = replicationUpdate.getMergePolicy(); String mapName = replicationUpdate.getMapName(); MapOperationProvider operationProvider = mapServiceContext.getMapOperationProvider(mapName); - MapOperation operation; - if (mergePolicy instanceof SplitBrainMergePolicy) { - SerializationService serializationService = nodeEngine.getSerializationService(); - MapMergeTypes mergingEntry = createMergingEntry(serializationService, replicationUpdate.getEntryView()); - //noinspection unchecked - operation = operationProvider.createMergeOperation(mapName, mergingEntry, - (SplitBrainMergePolicy) mergePolicy, true); - } else { - EntryView entryView = replicationUpdate.getEntryView(); - operation = operationProvider.createLegacyMergeOperation(mapName, entryView, (MapMergePolicy) mergePolicy, true); - } + SerializationService serializationService = nodeEngine.getSerializationService(); + MapMergeTypes mergingEntry = createMergingEntry(serializationService, replicationUpdate.getEntryView()); + //noinspection unchecked + MapOperation operation = operationProvider.createMergeOperation(mapName, mergingEntry, + (SplitBrainMergePolicy) mergePolicy, true); + try { int partitionId = nodeEngine.getPartitionService().getPartitionId(replicationUpdate.getEntryView().getKey()); Future future = nodeEngine.getOperationService() diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/MapServiceContext.java b/hazelcast/src/main/java/com/hazelcast/map/impl/MapServiceContext.java index 8246e0aeea135..92b51efa33f1e 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/MapServiceContext.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/MapServiceContext.java @@ -34,7 +34,6 @@ import com.hazelcast.map.impl.record.Record; import com.hazelcast.map.impl.recordstore.RecordStore; import com.hazelcast.map.impl.recordstore.RecordStoreMutationObserver; -import com.hazelcast.map.merge.MergePolicyProvider; import com.hazelcast.monitor.impl.LocalMapStatsImpl; import com.hazelcast.nio.serialization.Data; import com.hazelcast.partition.PartitioningStrategy; @@ -140,10 +139,6 @@ void removeRecordStoresFromPartitionMatchingWith(Predicate predicat NodeEngine getNodeEngine(); - MergePolicyProvider getMergePolicyProvider(); - - Object getMergePolicy(String name); - MapEventPublisher getMapEventPublisher(); MapEventJournal getEventJournal(); diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/MapServiceContextImpl.java b/hazelcast/src/main/java/com/hazelcast/map/impl/MapServiceContextImpl.java index ad6a5ad9be62f..2100ecc85841b 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/MapServiceContextImpl.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/MapServiceContextImpl.java @@ -19,7 +19,6 @@ import com.hazelcast.cluster.ClusterState; import com.hazelcast.config.InMemoryFormat; import com.hazelcast.config.MapConfig; -import com.hazelcast.config.MergePolicyConfig; import com.hazelcast.config.MetadataPolicy; import com.hazelcast.config.PartitioningStrategyConfig; import com.hazelcast.internal.eviction.ExpirationManager; @@ -68,7 +67,6 @@ import com.hazelcast.map.impl.recordstore.RecordStore; import com.hazelcast.map.impl.recordstore.RecordStoreMutationObserver; import com.hazelcast.map.listener.MapPartitionLostListener; -import com.hazelcast.map.merge.MergePolicyProvider; import com.hazelcast.monitor.impl.LocalMapStatsImpl; import com.hazelcast.nio.serialization.Data; import com.hazelcast.nio.serialization.DataType; @@ -137,7 +135,6 @@ class MapServiceContextImpl implements MapServiceContext { private final QueryCacheContext queryCacheContext; private final ExpirationManager expirationManager; private final PartitionScanRunner partitionScanRunner; - private final MergePolicyProvider mergePolicyProvider; private final MapNearCacheManager mapNearCacheManager; private final MapOperationProviders operationProviders; private final PartitionContainer[] partitionContainers; @@ -172,7 +169,6 @@ class MapServiceContextImpl implements MapServiceContext { this.expirationManager = new ExpirationManager(clearExpiredRecordsTask, nodeEngine); this.mapNearCacheManager = createMapNearCacheManager(); this.localMapStatsProvider = createLocalMapStatsProvider(); - this.mergePolicyProvider = new MergePolicyProvider(nodeEngine); this.mapEventPublisher = createMapEventPublisherSupport(); this.eventJournal = createEventJournal(); this.queryOptimizer = newOptimizer(nodeEngine.getProperties()); @@ -493,18 +489,6 @@ public NodeEngine getNodeEngine() { return nodeEngine; } - @Override - public MergePolicyProvider getMergePolicyProvider() { - return mergePolicyProvider; - } - - @Override - public Object getMergePolicy(String name) { - MapContainer mapContainer = getMapContainer(name); - MergePolicyConfig mergePolicyConfig = mapContainer.getMapConfig().getMergePolicyConfig(); - return mergePolicyProvider.getMergePolicy(mergePolicyConfig.getPolicy()); - } - @Override public MapEventPublisher getMapEventPublisher() { return mapEventPublisher; diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/MapSplitBrainHandlerService.java b/hazelcast/src/main/java/com/hazelcast/map/impl/MapSplitBrainHandlerService.java index 1ab33dd3d0611..37095e143d221 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/MapSplitBrainHandlerService.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/MapSplitBrainHandlerService.java @@ -18,7 +18,6 @@ import com.hazelcast.map.impl.recordstore.DefaultRecordStore; import com.hazelcast.map.impl.recordstore.RecordStore; -import com.hazelcast.map.merge.IgnoreMergingEntryMapMergePolicy; import com.hazelcast.spi.impl.merge.AbstractSplitBrainHandlerService; import com.hazelcast.spi.merge.DiscardMergePolicy; @@ -77,7 +76,8 @@ protected boolean hasEntries(RecordStore store) { @Override protected boolean hasMergeablePolicy(RecordStore store) { - Object mergePolicy = mapServiceContext.getMergePolicy(store.getName()); - return !(mergePolicy instanceof DiscardMergePolicy || mergePolicy instanceof IgnoreMergingEntryMapMergePolicy); + String policy = store.getMapContainer().getMapConfig().getMergePolicyConfig().getPolicy(); + Object mergePolicy = mapServiceContext.getNodeEngine().getSplitBrainMergePolicyProvider().getMergePolicy(policy); + return !(mergePolicy instanceof DiscardMergePolicy); } } diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/event/MapEventPublisherImpl.java b/hazelcast/src/main/java/com/hazelcast/map/impl/event/MapEventPublisherImpl.java index ef5f61040cf88..edd8605aae36a 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/event/MapEventPublisherImpl.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/event/MapEventPublisherImpl.java @@ -34,6 +34,7 @@ import com.hazelcast.spi.EventService; import com.hazelcast.spi.NodeEngine; import com.hazelcast.spi.impl.eventservice.impl.TrueEventFilter; +import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.partition.IPartitionService; import com.hazelcast.spi.properties.HazelcastProperty; import com.hazelcast.wan.ReplicationEventObject; @@ -98,7 +99,7 @@ public void publishWanUpdate(String mapName, } MapContainer mapContainer = mapServiceContext.getMapContainer(mapName); - Object wanMergePolicy = mapContainer.getWanMergePolicy(); + SplitBrainMergePolicy wanMergePolicy = mapContainer.getWanMergePolicy(); MapReplicationUpdate event = new MapReplicationUpdate(mapName, wanMergePolicy, entryView); publishWanEvent(mapName, event); } diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/BasePutOperation.java b/hazelcast/src/main/java/com/hazelcast/map/impl/operation/BasePutOperation.java index b654d640e84e7..da0a756373072 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/BasePutOperation.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/operation/BasePutOperation.java @@ -30,7 +30,6 @@ public abstract class BasePutOperation extends LockAwareOperation implements BackupAwareOperation { protected transient Object oldValue; - protected transient Data dataMergingValue; protected transient EntryEventType eventType; protected transient boolean putTransient; @@ -48,9 +47,10 @@ public BasePutOperation() { @Override protected void afterRunInternal() { mapServiceContext.interceptAfterPut(name, dataValue); - Object value = isPostProcessing(recordStore) ? recordStore.getRecord(dataKey).getValue() : dataValue; + Object value = isPostProcessing(recordStore) + ? recordStore.getRecord(dataKey).getValue() : dataValue; mapEventPublisher.publishEvent(getCallerAddress(), name, getEventType(), - dataKey, oldValue, value, dataMergingValue); + dataKey, oldValue, value); invalidateNearCache(dataKey); publishWanUpdate(dataKey, value); evict(dataKey); diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/DefaultMapOperationProvider.java b/hazelcast/src/main/java/com/hazelcast/map/impl/operation/DefaultMapOperationProvider.java index d8ee8eaad0608..ef13842ff221c 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/DefaultMapOperationProvider.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/operation/DefaultMapOperationProvider.java @@ -16,7 +16,6 @@ package com.hazelcast.map.impl.operation; -import com.hazelcast.core.EntryView; import com.hazelcast.map.EntryProcessor; import com.hazelcast.map.impl.MapEntries; import com.hazelcast.map.impl.query.Query; @@ -25,7 +24,6 @@ import com.hazelcast.map.impl.tx.TxnDeleteOperation; import com.hazelcast.map.impl.tx.TxnLockAndGetOperation; import com.hazelcast.map.impl.tx.TxnSetOperation; -import com.hazelcast.map.merge.MapMergePolicy; import com.hazelcast.nio.serialization.Data; import com.hazelcast.query.Predicate; import com.hazelcast.spi.impl.operationservice.Operation; @@ -193,12 +191,6 @@ public MapOperation createTxnSetOperation(String name, Data dataKey, Data value, return new TxnSetOperation(name, dataKey, value, version, ttl); } - @Override - public MapOperation createLegacyMergeOperation(String name, EntryView mergingEntry, - MapMergePolicy policy, boolean disableWanReplicationEvent) { - return new LegacyMergeOperation(name, mergingEntry, policy, disableWanReplicationEvent); - } - @Override public MapOperation createMergeOperation(String name, MapMergeTypes mergingValue, SplitBrainMergePolicy mergePolicy, diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/LegacyMergeOperation.java b/hazelcast/src/main/java/com/hazelcast/map/impl/operation/LegacyMergeOperation.java deleted file mode 100644 index bc2dc7efc31e1..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/LegacyMergeOperation.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.impl.operation; - -import com.hazelcast.core.EntryEventType; -import com.hazelcast.core.EntryView; -import com.hazelcast.map.impl.MapDataSerializerHook; -import com.hazelcast.map.impl.record.Record; -import com.hazelcast.map.merge.MapMergePolicy; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.Data; -import com.hazelcast.spi.impl.operationservice.Operation; - -import java.io.IOException; - -/** - * Contains a merging entry for split-brain healing with a {@link MapMergePolicy}. - */ -public class LegacyMergeOperation extends BasePutOperation { - - private MapMergePolicy mergePolicy; - private EntryView mergingEntry; - - private transient boolean merged; - - public LegacyMergeOperation() { - } - - public LegacyMergeOperation(String name, EntryView mergingEntry, MapMergePolicy policy, - boolean disableWanReplicationEvent) { - super(name, mergingEntry.getKey(), null); - this.mergingEntry = mergingEntry; - this.mergePolicy = policy; - this.disableWanReplicationEvent = disableWanReplicationEvent; - } - - @Override - protected void runInternal() { - Record oldRecord = recordStore.getRecord(dataKey); - if (oldRecord != null) { - oldValue = mapServiceContext.toData(oldRecord.getValue()); - } - merged = recordStore.merge(dataKey, mergingEntry, mergePolicy, getCallerProvenance()); - if (merged) { - Record record = recordStore.getRecord(dataKey); - if (record != null) { - dataValue = mapServiceContext.toData(record.getValue()); - dataMergingValue = mapServiceContext.toData(mergingEntry.getValue()); - } - } - } - - @Override - public Object getResponse() { - return merged; - } - - @Override - public boolean shouldBackup() { - return merged; - } - - @Override - protected void afterRunInternal() { - if (merged) { - eventType = EntryEventType.MERGED; - super.afterRunInternal(); - } - } - - @Override - public Operation getBackupOperation() { - if (dataValue == null) { - return new RemoveBackupOperation(name, dataKey, disableWanReplicationEvent); - } else { - return super.getBackupOperation(); - } - } - - @Override - protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); - out.writeObject(mergingEntry); - out.writeObject(mergePolicy); - out.writeBoolean(disableWanReplicationEvent); - } - - @Override - protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); - mergingEntry = in.readObject(); - mergePolicy = in.readObject(); - disableWanReplicationEvent = in.readBoolean(); - } - - @Override - public int getClassId() { - return MapDataSerializerHook.LEGACY_MERGE; - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/MapOperationProvider.java b/hazelcast/src/main/java/com/hazelcast/map/impl/operation/MapOperationProvider.java index 3c724af261e89..4c4bacfd2883f 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/MapOperationProvider.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/operation/MapOperationProvider.java @@ -16,12 +16,10 @@ package com.hazelcast.map.impl.operation; -import com.hazelcast.core.EntryView; import com.hazelcast.map.EntryProcessor; import com.hazelcast.map.IMap; import com.hazelcast.map.impl.MapEntries; import com.hazelcast.map.impl.query.Query; -import com.hazelcast.map.merge.MapMergePolicy; import com.hazelcast.nio.serialization.Data; import com.hazelcast.query.Predicate; import com.hazelcast.spi.impl.operationservice.Operation; @@ -111,11 +109,9 @@ MapOperation createTxnLockAndGetOperation(String name, Data dataKey, long timeou MapOperation createTxnSetOperation(String name, Data dataKey, Data value, long version, long ttl); - MapOperation createLegacyMergeOperation(String name, EntryView entryView, MapMergePolicy policy, - boolean disableWanReplicationEvent); - MapOperation createMergeOperation(String name, MapMergeTypes mergingValue, - SplitBrainMergePolicy mergePolicy, boolean disableWanReplicationEvent); + SplitBrainMergePolicy mergePolicy, + boolean disableWanReplicationEvent); MapOperation createMapFlushOperation(String name); diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/MergeOperation.java b/hazelcast/src/main/java/com/hazelcast/map/impl/operation/MergeOperation.java index a3a225ec015a8..c2246e27346dc 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/operation/MergeOperation.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/operation/MergeOperation.java @@ -59,8 +59,9 @@ public class MergeOperation extends MapOperation implements PartitionAwareOperat public MergeOperation() { } - MergeOperation(String name, List mergingEntries, SplitBrainMergePolicy mergePolicy, - boolean disableWanReplicationEvent) { + public MergeOperation(String name, List mergingEntries, + SplitBrainMergePolicy mergePolicy, + boolean disableWanReplicationEvent) { super(name); this.mergingEntries = mergingEntries; this.mergePolicy = mergePolicy; @@ -91,7 +92,6 @@ private void merge(MapMergeTypes mergingEntry) { Data dataKey = mergingEntry.getKey(); Data oldValue = hasMapListener ? getValue(dataKey) : null; - //noinspection unchecked if (recordStore.merge(mergingEntry, mergePolicy, getCallerProvenance())) { hasMergedValues = true; Data dataValue = getValueOrPostProcessedValue(dataKey, getValue(dataKey)); diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/recordstore/DefaultRecordStore.java b/hazelcast/src/main/java/com/hazelcast/map/impl/recordstore/DefaultRecordStore.java index 50e28f0244305..5b22bc3ef75a4 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/recordstore/DefaultRecordStore.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/recordstore/DefaultRecordStore.java @@ -19,11 +19,9 @@ import com.hazelcast.config.InMemoryFormat; import com.hazelcast.config.NativeMemoryConfig; import com.hazelcast.core.EntryEventType; -import com.hazelcast.core.EntryView; import com.hazelcast.cp.internal.datastructures.unsafe.lock.LockService; import com.hazelcast.logging.ILogger; import com.hazelcast.map.EntryLoader.MetadataAwareValue; -import com.hazelcast.map.impl.EntryViews; import com.hazelcast.map.impl.MapContainer; import com.hazelcast.map.impl.MapEntries; import com.hazelcast.map.impl.MapKeyLoader; @@ -41,7 +39,6 @@ import com.hazelcast.map.impl.querycache.publisher.PublisherRegistry; import com.hazelcast.map.impl.record.Record; import com.hazelcast.map.impl.record.Records; -import com.hazelcast.map.merge.MapMergePolicy; import com.hazelcast.nio.Address; import com.hazelcast.nio.serialization.Data; import com.hazelcast.query.impl.Index; @@ -74,7 +71,6 @@ import static com.hazelcast.core.EntryEventType.ADDED; import static com.hazelcast.core.EntryEventType.LOADED; import static com.hazelcast.core.EntryEventType.UPDATED; -import static com.hazelcast.map.impl.EntryViews.toLazyEntryView; import static com.hazelcast.map.impl.ExpirationTimeSetter.setExpirationTimes; import static com.hazelcast.map.impl.mapstore.MapDataStores.EMPTY_MAP_DATA_STORE; import static com.hazelcast.spi.impl.merge.MergingValueFactory.createMergingEntry; @@ -323,7 +319,7 @@ public String getLockOwnerInfo(Data key) { @Override public Record loadRecordOrNull(Data key, boolean backup, Address callerAddress) { - Record record = null; + Record record; long ttl = DEFAULT_TTL; Object value = mapDataStore.load(key); if (value == null) { @@ -348,9 +344,10 @@ record = createRecord(key, value, ttl, DEFAULT_MAX_IDLE, getNow()); key, null, value, null); } evictEntries(key); - // here, we are only publishing events for loaded entries. This is required for notifying query-caches + // here, we are only publishing events for loaded + // entries. This is required for notifying query-caches // otherwise query-caches cannot see loaded entries - if (!backup && record != null && hasQueryCache()) { + if (!backup && hasQueryCache()) { addEventToQueryCache(record); } return record; @@ -513,6 +510,7 @@ public boolean remove(Data key, Object testValue) { removeIndex(record); mapDataStore.remove(key, now); onStore(record); + // TODO record.getKey() may produce NPE mutationObserver.onRemoveRecord(record.getKey(), record); storage.removeRecord(record); removed = true; @@ -799,67 +797,6 @@ record = createRecord(key, newValue, DEFAULT_TTL, DEFAULT_MAX_IDLE, now); return newValue != null; } - @Override - public boolean merge(Data key, EntryView mergingEntry, MapMergePolicy mergePolicy) { - return merge(key, mergingEntry, mergePolicy, CallerProvenance.NOT_WAN); - } - - @Override - public boolean merge(Data key, EntryView mergingEntry, - MapMergePolicy mergePolicy, CallerProvenance provenance) { - checkIfLoaded(); - long now = getNow(); - - Record record = getRecordOrNull(key, now, false); - mergingEntry = toLazyEntryView(mergingEntry, serializationService, mergePolicy); - Object newValue; - Object oldValue = null; - if (record == null) { - EntryView nullEntryView = EntryViews.createLazyNullEntryView(key, serializationService); - newValue = mergePolicy.merge(name, mergingEntry, nullEntryView); - if (newValue == null) { - return false; - } - - record = createRecord(key, newValue, DEFAULT_TTL, DEFAULT_MAX_IDLE, now); - mergeRecordExpiration(record, mergingEntry); - newValue = persistenceEnabledFor(provenance) - ? mapDataStore.add(key, newValue, record.getExpirationTime(), now) : newValue; - recordFactory.setValue(record, newValue); - storage.put(key, record); - mutationObserver.onPutRecord(key, record); - } else { - oldValue = record.getValue(); - EntryView existingEntry = EntryViews.createLazyEntryView(record.getKey(), record.getValue(), - record, serializationService, mergePolicy); - newValue = mergePolicy.merge(name, mergingEntry, existingEntry); - // existing entry will be removed - if (newValue == null) { - removeIndex(record); - if (persistenceEnabledFor(provenance)) { - mapDataStore.remove(key, now); - } - onStore(record); - mutationObserver.onRemoveRecord(key, record); - storage.removeRecord(record); - return true; - } - // same with the existing entry so no need to map-store etc operations. - if (valueComparator.isEqual(newValue, oldValue, serializationService)) { - mergeRecordExpiration(record, mergingEntry); - return true; - } - - newValue = persistenceEnabledFor(provenance) - ? mapDataStore.add(key, newValue, record.getExpirationTime(), now) : newValue; - onStore(record); - mutationObserver.onUpdateRecord(key, record, newValue); - storage.updateRecordValue(key, record, newValue); - } - saveIndex(record, oldValue); - return newValue != null; - } - @Override public Object replace(Data key, Object update) { checkIfLoaded(); diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/recordstore/RecordStore.java b/hazelcast/src/main/java/com/hazelcast/map/impl/recordstore/RecordStore.java index b1da60669b7fd..10e711b2b5bcd 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/recordstore/RecordStore.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/recordstore/RecordStore.java @@ -17,11 +17,10 @@ package com.hazelcast.map.impl.recordstore; import com.hazelcast.config.InMemoryFormat; -import com.hazelcast.core.EntryView; -import com.hazelcast.map.IMap; import com.hazelcast.internal.eviction.ExpiredKey; import com.hazelcast.internal.nearcache.impl.invalidation.InvalidationQueue; import com.hazelcast.internal.util.comparators.ValueComparator; +import com.hazelcast.map.IMap; import com.hazelcast.map.MapLoader; import com.hazelcast.map.impl.MapContainer; import com.hazelcast.map.impl.MapEntries; @@ -30,7 +29,6 @@ import com.hazelcast.map.impl.mapstore.MapDataStore; import com.hazelcast.map.impl.record.Record; import com.hazelcast.map.impl.record.RecordFactory; -import com.hazelcast.map.merge.MapMergePolicy; import com.hazelcast.monitor.LocalRecordStoreStats; import com.hazelcast.nio.Address; import com.hazelcast.nio.serialization.Data; @@ -234,20 +232,6 @@ boolean merge(MapMergeTypes mergingEntry, SplitBrainMergePolicy mergePolicy, CallerProvenance provenance); - boolean merge(Data dataKey, EntryView mergingEntry, MapMergePolicy mergePolicy); - - /** - * Merges the given {@link EntryView} via the given {@link MapMergePolicy}. - * - * @param dataKey the key to be merged - * @param mergingEntry the {@link EntryView} instance to merge - * @param mergePolicy the {@link MapMergePolicy} instance to apply - * @param provenance origin of call to this method. - * @return {@code true} if merge is applied, otherwise {@code false} - */ - boolean merge(Data dataKey, EntryView mergingEntry, MapMergePolicy mergePolicy, - CallerProvenance provenance); - R getRecord(Data key); /** diff --git a/hazelcast/src/main/java/com/hazelcast/map/impl/wan/MapReplicationUpdate.java b/hazelcast/src/main/java/com/hazelcast/map/impl/wan/MapReplicationUpdate.java index 49a44b8ff320a..f9545ec697ca6 100644 --- a/hazelcast/src/main/java/com/hazelcast/map/impl/wan/MapReplicationUpdate.java +++ b/hazelcast/src/main/java/com/hazelcast/map/impl/wan/MapReplicationUpdate.java @@ -21,8 +21,9 @@ import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.Data; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.wan.ReplicationEventObject; +import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.wan.DistributedServiceWanEventCounters; +import com.hazelcast.wan.ReplicationEventObject; import com.hazelcast.wan.impl.WanDataSerializerHook; import java.io.IOException; @@ -32,16 +33,20 @@ */ public class MapReplicationUpdate implements ReplicationEventObject, IdentifiedDataSerializable { private String mapName; - /** The policy how to merge the entry on the receiving cluster */ - private Object mergePolicy; - /** The updated entry */ + /** + * The policy how to merge the entry on the receiving cluster + */ + private SplitBrainMergePolicy mergePolicy; + /** + * The updated entry + */ private WanMapEntryView entryView; public MapReplicationUpdate() { } public MapReplicationUpdate(String mapName, - Object mergePolicy, + SplitBrainMergePolicy mergePolicy, EntryView entryView) { this.mergePolicy = mergePolicy; this.mapName = mapName; @@ -60,11 +65,11 @@ public void setMapName(String mapName) { this.mapName = mapName; } - public Object getMergePolicy() { + public SplitBrainMergePolicy getMergePolicy() { return mergePolicy; } - public void setMergePolicy(Object mergePolicy) { + public void setMergePolicy(SplitBrainMergePolicy mergePolicy) { this.mergePolicy = mergePolicy; } diff --git a/hazelcast/src/main/java/com/hazelcast/map/merge/HigherHitsMapMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/map/merge/HigherHitsMapMergePolicy.java deleted file mode 100644 index cfe1e24dd9bda..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/merge/HigherHitsMapMergePolicy.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.core.EntryView; -import com.hazelcast.map.impl.MapDataSerializerHook; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.IdentifiedDataSerializable; - -import java.io.IOException; - - -/** - * Merges map entries from source to destination map if the source entry - * has more hits than the destination one. - */ -public class HigherHitsMapMergePolicy implements MapMergePolicy, IdentifiedDataSerializable { - - @Override - public Object merge(String mapName, EntryView mergingEntry, EntryView existingEntry) { - if (mergingEntry.getHits() >= existingEntry.getHits()) { - return mergingEntry.getValue(); - } - return existingEntry.getValue(); - } - - @Override - public void writeData(ObjectDataOutput out) throws IOException { - } - - @Override - public void readData(ObjectDataInput in) throws IOException { - } - - @Override - public int getFactoryId() { - return MapDataSerializerHook.F_ID; - } - - @Override - public int getClassId() { - return MapDataSerializerHook.HIGHER_HITS_MERGE_POLICY; - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/map/merge/IgnoreMergingEntryMapMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/map/merge/IgnoreMergingEntryMapMergePolicy.java deleted file mode 100644 index 29edf8d12256a..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/merge/IgnoreMergingEntryMapMergePolicy.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.core.EntryView; -import com.hazelcast.map.impl.MapDataSerializerHook; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.IdentifiedDataSerializable; - -import java.io.IOException; - -/** - * Ignores the merging entry while collecting merge-needed entries on merging side. - *

- * Note: Its operations are not implemented since it is only used as a marker. - * - * @since 3.9 - */ -public final class IgnoreMergingEntryMapMergePolicy implements MapMergePolicy, IdentifiedDataSerializable { - - public IgnoreMergingEntryMapMergePolicy() { - } - - @Override - public Object merge(String mapName, EntryView mergingEntry, EntryView existingEntry) { - throw new UnsupportedOperationException(); - } - - @Override - public void writeData(ObjectDataOutput out) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public void readData(ObjectDataInput in) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public int getFactoryId() { - return MapDataSerializerHook.F_ID; - } - - @Override - public int getClassId() { - throw new UnsupportedOperationException(); - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/map/merge/LatestUpdateMapMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/map/merge/LatestUpdateMapMergePolicy.java deleted file mode 100644 index 99a5c31d933aa..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/merge/LatestUpdateMapMergePolicy.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.core.EntryView; -import com.hazelcast.map.impl.MapDataSerializerHook; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.IdentifiedDataSerializable; - -import java.io.IOException; - -/** - * Merges map entries from source to destination map if the source entry - * was updated more recently than the destination entry. - *

- * Note: This policy can only be used if the clocks of the nodes are in sync. - */ -public class LatestUpdateMapMergePolicy implements MapMergePolicy, IdentifiedDataSerializable { - - @Override - public Object merge(String mapName, EntryView mergingEntry, EntryView existingEntry) { - if (mergingEntry.getLastUpdateTime() >= existingEntry.getLastUpdateTime()) { - return mergingEntry.getValue(); - } - return existingEntry.getValue(); - } - - @Override - public void writeData(ObjectDataOutput out) throws IOException { - } - - @Override - public void readData(ObjectDataInput in) throws IOException { - } - - @Override - public int getFactoryId() { - return MapDataSerializerHook.F_ID; - } - - @Override - public int getClassId() { - return MapDataSerializerHook.LATEST_UPDATE_MERGE_POLICY; - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/map/merge/MapMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/map/merge/MapMergePolicy.java deleted file mode 100644 index ebecf2405f0ec..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/merge/MapMergePolicy.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.core.EntryView; -import com.hazelcast.nio.serialization.DataSerializable; - -/** - * Policy for merging map entries after a split-brain has been healed. - * - * @see com.hazelcast.map.merge.HigherHitsMapMergePolicy - * @see com.hazelcast.map.merge.LatestUpdateMapMergePolicy - * @see com.hazelcast.map.merge.PassThroughMergePolicy - * @see com.hazelcast.map.merge.PutIfAbsentMapMergePolicy - */ -public interface MapMergePolicy extends DataSerializable { - - /** - * Selects one of the merging and existing map entries to be merged. - *

- * Note that the {@code existingEntry} may be {@code null} if there - * is no entry with the same key in the destination map. - * This happens, when the entry for that key was - *

    - *
  • only created in the smaller sub-cluster during the split-brain
  • - *
  • removed in the larger sub-cluster during the split-brain
  • - *
- * - * @param mapName name of the map - * @param mergingEntry {@link EntryView} instance that has the map entry to be merged - * @param existingEntry {@link EntryView} instance that has the existing map entry - * or {@code null} if there is no existing map entry - * @return the selected value for merging or {@code null} if the entry should be removed - */ - Object merge(String mapName, EntryView mergingEntry, EntryView existingEntry); -} diff --git a/hazelcast/src/main/java/com/hazelcast/map/merge/MergePolicyProvider.java b/hazelcast/src/main/java/com/hazelcast/map/merge/MergePolicyProvider.java deleted file mode 100644 index dff0ac0b72a33..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/merge/MergePolicyProvider.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.config.InvalidConfigurationException; -import com.hazelcast.spi.NodeEngine; -import com.hazelcast.spi.merge.SplitBrainMergePolicy; -import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; -import com.hazelcast.util.ConstructorFunction; - -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import static com.hazelcast.nio.ClassLoaderUtil.newInstance; -import static com.hazelcast.util.ConcurrencyUtil.getOrPutIfAbsent; - -/** - * A provider for {@link com.hazelcast.map.merge.MapMergePolicy} instances. - */ -public final class MergePolicyProvider { - - private final ConcurrentMap mergePolicyMap = new ConcurrentHashMap<>(); - - private final ConstructorFunction policyConstructorFunction - = new ConstructorFunction() { - @Override - public MapMergePolicy createNew(String className) { - try { - return newInstance(nodeEngine.getConfigClassLoader(), className); - } catch (Exception e) { - throw new InvalidConfigurationException("Invalid MapMergePolicy: " + className, e); - } - } - }; - - private final NodeEngine nodeEngine; - private final SplitBrainMergePolicyProvider policyProvider; - - public MergePolicyProvider(NodeEngine nodeEngine) { - this.nodeEngine = nodeEngine; - this.policyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); - addOutOfBoxPolicies(); - } - - private void addOutOfBoxPolicies() { - mergePolicyMap.put(PutIfAbsentMapMergePolicy.class.getName(), new PutIfAbsentMapMergePolicy()); - mergePolicyMap.put(HigherHitsMapMergePolicy.class.getName(), new HigherHitsMapMergePolicy()); - mergePolicyMap.put(PassThroughMergePolicy.class.getName(), new PassThroughMergePolicy()); - mergePolicyMap.put(LatestUpdateMapMergePolicy.class.getName(), new LatestUpdateMapMergePolicy()); - } - - /** - * Returns an instance of a merge policy by its classname. - *

- * First tries to resolve the classname as {@link SplitBrainMergePolicy}, - * then as {@link com.hazelcast.map.merge.MapMergePolicy}. - *

- * If no merge policy matches an exception is thrown. - * - * @param className the classname of the given merge policy - * @return an instance of the merge policy class - */ - public Object getMergePolicy(String className) { - if (className == null) { - throw new InvalidConfigurationException("Class name is mandatory!"); - } - try { - return policyProvider.getMergePolicy(className); - } catch (InvalidConfigurationException e) { - return getOrPutIfAbsent(mergePolicyMap, className, policyConstructorFunction); - } - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/map/merge/PassThroughMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/map/merge/PassThroughMergePolicy.java deleted file mode 100644 index 324b3c272c955..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/merge/PassThroughMergePolicy.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.core.EntryView; -import com.hazelcast.map.impl.MapDataSerializerHook; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.IdentifiedDataSerializable; - -import java.io.IOException; - -/** - * Merges map entries from source to destination directly unless the merging entry is {@code null}. - */ -public class PassThroughMergePolicy implements MapMergePolicy, IdentifiedDataSerializable { - - @Override - public Object merge(String mapName, EntryView mergingEntry, EntryView existingEntry) { - return mergingEntry == null ? existingEntry.getValue() : mergingEntry.getValue(); - } - - @Override - public void writeData(ObjectDataOutput out) throws IOException { - } - - @Override - public void readData(ObjectDataInput in) throws IOException { - } - - @Override - public int getFactoryId() { - return MapDataSerializerHook.F_ID; - } - - @Override - public int getClassId() { - return MapDataSerializerHook.PASS_THROUGH_MERGE_POLICY; - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/map/merge/PutIfAbsentMapMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/map/merge/PutIfAbsentMapMergePolicy.java deleted file mode 100644 index da8562ca4af0d..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/merge/PutIfAbsentMapMergePolicy.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.core.EntryView; -import com.hazelcast.map.impl.MapDataSerializerHook; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.IdentifiedDataSerializable; - -import java.io.IOException; - -/** - * Merges map entries from source to destination if they don't exist in the destination map. - */ -public class PutIfAbsentMapMergePolicy implements MapMergePolicy, IdentifiedDataSerializable { - - @Override - public Object merge(String mapName, EntryView mergingEntry, EntryView existingEntry) { - if (existingEntry.getValue() == null) { - return mergingEntry.getValue(); - } - return existingEntry.getValue(); - } - - @Override - public void writeData(ObjectDataOutput out) throws IOException { - } - - @Override - public void readData(ObjectDataInput in) throws IOException { - } - - @Override - public int getFactoryId() { - return MapDataSerializerHook.F_ID; - } - - @Override - public int getClassId() { - return MapDataSerializerHook.PUT_IF_ABSENT_MERGE_POLICY; - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/map/merge/package-info.java b/hazelcast/src/main/java/com/hazelcast/map/merge/package-info.java deleted file mode 100644 index b9cf3653fe0ab..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/map/merge/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Contains out-of-the-box merge policies for {@link com.hazelcast.map.IMap}. - */ -package com.hazelcast.map.merge; diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/ReplicatedMapMergeRunnable.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/ReplicatedMapMergeRunnable.java index 68c5f2fa59290..5593887ffe8d4 100644 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/ReplicatedMapMergeRunnable.java +++ b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/ReplicatedMapMergeRunnable.java @@ -19,19 +19,14 @@ import com.hazelcast.config.InMemoryFormat; import com.hazelcast.config.MergePolicyConfig; import com.hazelcast.config.ReplicatedMapConfig; -import com.hazelcast.replicatedmap.impl.operation.LegacyMergeOperation; import com.hazelcast.replicatedmap.impl.operation.MergeOperationFactory; -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; import com.hazelcast.replicatedmap.impl.record.ReplicatedRecord; import com.hazelcast.replicatedmap.impl.record.ReplicatedRecordStore; -import com.hazelcast.replicatedmap.merge.ReplicatedMapMergePolicy; import com.hazelcast.spi.NodeEngine; -import com.hazelcast.spi.impl.operationservice.Operation; -import com.hazelcast.spi.impl.operationservice.OperationFactory; import com.hazelcast.spi.impl.merge.AbstractMergeRunnable; +import com.hazelcast.spi.impl.operationservice.OperationFactory; import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergeTypes.ReplicatedMapMergeTypes; -import com.hazelcast.spi.serialization.SerializationService; import java.util.Collection; import java.util.Iterator; @@ -67,34 +62,6 @@ protected void mergeStore(ReplicatedRecordStore store, BiConsumer consumer) { - int partitionId = store.getPartitionId(); - String name = store.getName(); - ReplicatedMapMergePolicy mergePolicy = ((ReplicatedMapMergePolicy) getMergePolicy(name)); - - Iterator iterator = store.recordIterator(); - while (iterator.hasNext()) { - ReplicatedRecord record = iterator.next(); - - ReplicatedMapEntryView entryView = createEntryView(record, getSerializationService()); - LegacyMergeOperation operation = new LegacyMergeOperation(name, record.getKeyInternal(), entryView, mergePolicy); - - consumer.accept(partitionId, operation); - } - } - - private static ReplicatedMapEntryView createEntryView(ReplicatedRecord record, SerializationService ss) { - return new ReplicatedMapEntryView(ss) - .setKey(record.getKeyInternal()) - .setValue(record.getValueInternal()) - .setHits(record.getHits()) - .setTtl(record.getTtlMillis()) - .setLastAccessTime(record.getLastAccessTime()) - .setCreationTime(record.getCreationTime()) - .setLastUpdateTime(record.getUpdateTime()); - } - @Override protected int getBatchSize(String dataStructureName) { ReplicatedMapConfig replicatedMapConfig = getReplicatedMapConfig(dataStructureName); @@ -109,8 +76,8 @@ protected InMemoryFormat getInMemoryFormat(String dataStructureName) { } @Override - protected Object getMergePolicy(String dataStructureName) { - return service.getMergePolicy(dataStructureName); + protected SplitBrainMergePolicy getMergePolicy(String dataStructureName) { + return mergePolicyProvider.getMergePolicy(dataStructureName); } @Override diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/ReplicatedMapService.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/ReplicatedMapService.java index f57dd8f3cf6ee..255b914acfe9f 100644 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/ReplicatedMapService.java +++ b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/ReplicatedMapService.java @@ -16,6 +16,8 @@ package com.hazelcast.replicatedmap.impl; +import com.hazelcast.cluster.Member; +import com.hazelcast.cluster.MemberSelector; import com.hazelcast.config.Config; import com.hazelcast.config.ListenerConfig; import com.hazelcast.config.MergePolicyConfig; @@ -23,8 +25,6 @@ import com.hazelcast.core.DistributedObject; import com.hazelcast.core.EntryListener; import com.hazelcast.core.HazelcastInstanceAware; -import com.hazelcast.cluster.Member; -import com.hazelcast.cluster.MemberSelector; import com.hazelcast.internal.cluster.ClusterService; import com.hazelcast.internal.partition.InternalPartition; import com.hazelcast.internal.partition.impl.InternalPartitionServiceImpl; @@ -38,20 +38,20 @@ import com.hazelcast.replicatedmap.impl.operation.CheckReplicaVersionOperation; import com.hazelcast.replicatedmap.impl.operation.ReplicationOperation; import com.hazelcast.replicatedmap.impl.record.ReplicatedRecordStore; -import com.hazelcast.replicatedmap.merge.MergePolicyProvider; import com.hazelcast.spi.EventPublishingService; import com.hazelcast.spi.ManagedService; -import com.hazelcast.spi.partition.MigrationAwareService; import com.hazelcast.spi.NodeEngine; -import com.hazelcast.spi.impl.operationservice.Operation; -import com.hazelcast.spi.impl.operationservice.OperationService; -import com.hazelcast.spi.partition.PartitionMigrationEvent; -import com.hazelcast.spi.partition.PartitionReplicationEvent; import com.hazelcast.spi.QuorumAwareService; import com.hazelcast.spi.RemoteService; import com.hazelcast.spi.SplitBrainHandlerService; import com.hazelcast.spi.StatisticsAwareService; import com.hazelcast.spi.impl.eventservice.impl.TrueEventFilter; +import com.hazelcast.spi.impl.operationservice.Operation; +import com.hazelcast.spi.impl.operationservice.OperationService; +import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; +import com.hazelcast.spi.partition.MigrationAwareService; +import com.hazelcast.spi.partition.PartitionMigrationEvent; +import com.hazelcast.spi.partition.PartitionReplicationEvent; import com.hazelcast.spi.serialization.SerializationService; import com.hazelcast.util.ConstructorFunction; import com.hazelcast.util.ContextMutexFactory; @@ -110,9 +110,9 @@ public Object createNew(String name) { private final ReplicatedMapEventPublishingService eventPublishingService; private final ReplicatedMapSplitBrainHandlerService splitBrainHandlerService; private final LocalReplicatedMapStatsProvider statsProvider; + private final SplitBrainMergePolicyProvider mergePolicyProvider; private ScheduledFuture antiEntropyFuture; - private MergePolicyProvider mergePolicyProvider; public ReplicatedMapService(NodeEngine nodeEngine) { this.nodeEngine = nodeEngine; @@ -124,7 +124,7 @@ public ReplicatedMapService(NodeEngine nodeEngine) { this.eventPublishingService = new ReplicatedMapEventPublishingService(this); this.splitBrainHandlerService = new ReplicatedMapSplitBrainHandlerService(this); this.quorumService = nodeEngine.getQuorumService(); - this.mergePolicyProvider = new MergePolicyProvider(nodeEngine); + this.mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); this.statsProvider = new LocalReplicatedMapStatsProvider(config, partitionContainers); } diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/operation/LegacyMergeOperation.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/operation/LegacyMergeOperation.java deleted file mode 100644 index 80b5858c627bb..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/operation/LegacyMergeOperation.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.impl.operation; - -import com.hazelcast.nio.IOUtil; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.replicatedmap.impl.ReplicatedMapService; -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; -import com.hazelcast.replicatedmap.impl.record.ReplicatedRecordStore; -import com.hazelcast.replicatedmap.merge.ReplicatedMapMergePolicy; - -import java.io.IOException; - -/** - * Contains a merging entry for split-brain healing with with a {@link ReplicatedMapMergePolicy}. - */ -public class LegacyMergeOperation extends AbstractNamedSerializableOperation { - - private String name; - private Object key; - private ReplicatedMapEntryView entryView; - private ReplicatedMapMergePolicy policy; - - public LegacyMergeOperation() { - } - - public LegacyMergeOperation(String name, Object key, ReplicatedMapEntryView entryView, ReplicatedMapMergePolicy policy) { - this.name = name; - this.key = key; - this.entryView = entryView; - this.policy = policy; - } - - @Override - public void run() throws Exception { - ReplicatedMapService service = getService(); - ReplicatedRecordStore store = service.getReplicatedRecordStore(name, true, key); - store.merge(key, entryView, policy); - } - - @Override - protected void writeInternal(ObjectDataOutput out) throws IOException { - out.writeUTF(name); - IOUtil.writeObject(out, key); - out.writeObject(entryView); - out.writeObject(policy); - } - - @Override - protected void readInternal(ObjectDataInput in) throws IOException { - name = in.readUTF(); - key = IOUtil.readObject(in); - entryView = in.readObject(); - policy = in.readObject(); - } - - @Override - public int getClassId() { - return ReplicatedMapDataSerializerHook.LEGACY_MERGE; - } - - @Override - public String getName() { - return name; - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/operation/ReplicatedMapDataSerializerHook.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/operation/ReplicatedMapDataSerializerHook.java index 8627964fa122e..1a6a659cce03b 100644 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/operation/ReplicatedMapDataSerializerHook.java +++ b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/operation/ReplicatedMapDataSerializerHook.java @@ -23,10 +23,6 @@ import com.hazelcast.nio.serialization.IdentifiedDataSerializable; import com.hazelcast.replicatedmap.impl.record.RecordMigrationInfo; import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; -import com.hazelcast.replicatedmap.merge.HigherHitsMapMergePolicy; -import com.hazelcast.replicatedmap.merge.LatestUpdateMapMergePolicy; -import com.hazelcast.replicatedmap.merge.PassThroughMergePolicy; -import com.hazelcast.replicatedmap.merge.PutIfAbsentMapMergePolicy; import com.hazelcast.util.ConstructorFunction; import static com.hazelcast.internal.serialization.impl.FactoryIdHelper.REPLICATED_MAP_DS_FACTORY; @@ -47,29 +43,24 @@ public class ReplicatedMapDataSerializerHook implements DataSerializerHook { public static final int PUT = 6; public static final int REMOVE = 7; public static final int SIZE = 8; - public static final int LEGACY_MERGE = 9; - public static final int VERSION_RESPONSE_PAIR = 10; - public static final int GET = 11; - public static final int CHECK_REPLICA_VERSION = 12; - public static final int CONTAINS_KEY = 13; - public static final int CONTAINS_VALUE = 14; - public static final int ENTRY_SET = 15; - public static final int EVICTION = 16; - public static final int IS_EMPTY = 17; - public static final int KEY_SET = 18; - public static final int REPLICATION = 19; - public static final int REQUEST_MAP_DATA = 20; - public static final int SYNC_REPLICATED_DATA = 21; - public static final int VALUES = 22; - public static final int CLEAR_OP_FACTORY = 23; - public static final int PUT_ALL_OP_FACTORY = 24; - public static final int RECORD_MIGRATION_INFO = 25; - public static final int HIGHER_HITS_MERGE_POLICY = 26; - public static final int LATEST_UPDATE_MERGE_POLICY = 27; - public static final int PASS_THROUGH_MERGE_POLICY = 28; - public static final int PUT_IF_ABSENT_MERGE_POLICY = 29; - public static final int MERGE_FACTORY = 30; - public static final int MERGE = 31; + public static final int VERSION_RESPONSE_PAIR = 9; + public static final int GET = 10; + public static final int CHECK_REPLICA_VERSION = 11; + public static final int CONTAINS_KEY = 12; + public static final int CONTAINS_VALUE = 13; + public static final int ENTRY_SET = 14; + public static final int EVICTION = 15; + public static final int IS_EMPTY = 16; + public static final int KEY_SET = 17; + public static final int REPLICATION = 18; + public static final int REQUEST_MAP_DATA = 19; + public static final int SYNC_REPLICATED_DATA = 20; + public static final int VALUES = 21; + public static final int CLEAR_OP_FACTORY = 22; + public static final int PUT_ALL_OP_FACTORY = 23; + public static final int RECORD_MIGRATION_INFO = 24; + public static final int MERGE_FACTORY = 25; + public static final int MERGE = 26; private static final int LEN = MERGE + 1; @@ -135,12 +126,6 @@ public IdentifiedDataSerializable createNew(Integer arg) { return new SizeOperation(); } }; - constructors[LEGACY_MERGE] = new ConstructorFunction() { - @Override - public IdentifiedDataSerializable createNew(Integer arg) { - return new LegacyMergeOperation(); - } - }; constructors[VERSION_RESPONSE_PAIR] = new ConstructorFunction() { @Override public IdentifiedDataSerializable createNew(Integer arg) { @@ -237,30 +222,6 @@ public IdentifiedDataSerializable createNew(Integer arg) { return new RecordMigrationInfo(); } }; - constructors[HIGHER_HITS_MERGE_POLICY] = new ConstructorFunction() { - @Override - public IdentifiedDataSerializable createNew(Integer arg) { - return HigherHitsMapMergePolicy.INSTANCE; - } - }; - constructors[LATEST_UPDATE_MERGE_POLICY] = new ConstructorFunction() { - @Override - public IdentifiedDataSerializable createNew(Integer arg) { - return LatestUpdateMapMergePolicy.INSTANCE; - } - }; - constructors[PASS_THROUGH_MERGE_POLICY] = new ConstructorFunction() { - @Override - public IdentifiedDataSerializable createNew(Integer arg) { - return PassThroughMergePolicy.INSTANCE; - } - }; - constructors[PUT_IF_ABSENT_MERGE_POLICY] = new ConstructorFunction() { - @Override - public IdentifiedDataSerializable createNew(Integer arg) { - return PutIfAbsentMapMergePolicy.INSTANCE; - } - }; constructors[MERGE_FACTORY] = new ConstructorFunction() { @Override public IdentifiedDataSerializable createNew(Integer arg) { diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/record/AbstractReplicatedRecordStore.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/record/AbstractReplicatedRecordStore.java index 9c957eaa19e90..810d9d27825be 100644 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/record/AbstractReplicatedRecordStore.java +++ b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/record/AbstractReplicatedRecordStore.java @@ -16,15 +16,14 @@ package com.hazelcast.replicatedmap.impl.record; -import com.hazelcast.cluster.memberselector.MemberSelectors; import com.hazelcast.cluster.Member; +import com.hazelcast.cluster.memberselector.MemberSelectors; import com.hazelcast.nio.Address; import com.hazelcast.nio.serialization.Data; import com.hazelcast.replicatedmap.impl.ReplicatedMapEventPublishingService; import com.hazelcast.replicatedmap.impl.ReplicatedMapService; import com.hazelcast.replicatedmap.impl.operation.ReplicateUpdateOperation; import com.hazelcast.replicatedmap.impl.operation.VersionResponsePair; -import com.hazelcast.replicatedmap.merge.ReplicatedMapMergePolicy; import com.hazelcast.spi.impl.operationservice.OperationService; import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergeTypes.ReplicatedMapMergeTypes; @@ -377,54 +376,6 @@ record = buildReplicatedRecord(marshalledKey, newValue, 0); return true; } - @Override - @SuppressWarnings("unchecked") - public boolean merge(Object key, ReplicatedMapEntryView mergingEntry, ReplicatedMapMergePolicy mergePolicy) { - K marshalledKey = (K) marshall(key); - InternalReplicatedMapStorage storage = getStorage(); - ReplicatedRecord existingRecord = storage.get(marshalledKey); - if (existingRecord == null) { - ReplicatedMapEntryView nullEntryView - = new ReplicatedMapEntryView(serializationService).setKey(key); - V newValue = (V) mergePolicy.merge(name, mergingEntry, nullEntryView); - if (newValue == null) { - return false; - } - existingRecord = buildReplicatedRecord(marshalledKey, newValue, 0); - storage.put(marshalledKey, existingRecord); - storage.incrementVersion(); - Data dataKey = serializationService.toData(marshalledKey); - Data dataValue = serializationService.toData(newValue); - VersionResponsePair responsePair = new VersionResponsePair(mergingEntry.getValue(), getVersion()); - sendReplicationOperation(false, name, dataKey, dataValue, existingRecord.getTtlMillis(), responsePair); - } else { - ReplicatedMapEntryView existingEntry = new ReplicatedMapEntryView(serializationService) - .setKey(key) - .setValue(existingRecord.getValueInternal()) - .setCreationTime(existingRecord.getCreationTime()) - .setLastUpdateTime(existingRecord.getUpdateTime()) - .setLastAccessTime(existingRecord.getLastAccessTime()) - .setHits(existingRecord.getHits()) - .setTtl(existingRecord.getTtlMillis()); - V newValue = (V) mergePolicy.merge(name, mergingEntry, existingEntry); - if (newValue == null) { - storage.remove(marshalledKey, existingRecord); - storage.incrementVersion(); - Data dataKey = serializationService.toData(marshalledKey); - VersionResponsePair responsePair = new VersionResponsePair(mergingEntry.getValue(), getVersion()); - sendReplicationOperation(true, name, dataKey, null, existingRecord.getTtlMillis(), responsePair); - return false; - } - existingRecord.setValueInternal(newValue, existingRecord.getTtlMillis()); - storage.incrementVersion(); - Data dataKey = serializationService.toData(marshalledKey); - Data dataValue = serializationService.toData(newValue); - VersionResponsePair responsePair = new VersionResponsePair(mergingEntry.getValue(), getVersion()); - sendReplicationOperation(false, name, dataKey, dataValue, existingRecord.getTtlMillis(), responsePair); - } - return true; - } - private void sendReplicationOperation(boolean isRemove, String name, Data key, Data value, long ttl, VersionResponsePair response) { Collection members = nodeEngine.getClusterService().getMembers(MemberSelectors.DATA_MEMBER_SELECTOR); diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/record/ReplicatedRecordStore.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/record/ReplicatedRecordStore.java index c8ca69bbfcbf3..395a06fd4f353 100644 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/record/ReplicatedRecordStore.java +++ b/hazelcast/src/main/java/com/hazelcast/replicatedmap/impl/record/ReplicatedRecordStore.java @@ -16,7 +16,6 @@ package com.hazelcast.replicatedmap.impl.record; -import com.hazelcast.replicatedmap.merge.ReplicatedMapMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergeTypes.ReplicatedMapMergeTypes; import com.hazelcast.util.scheduler.ScheduledEntry; @@ -107,13 +106,4 @@ public interface ReplicatedRecordStore { * @return {@code true} if merge is applied, otherwise {@code false} */ boolean merge(ReplicatedMapMergeTypes mergingEntry, SplitBrainMergePolicy mergePolicy); - - /** - * Merges the given {@link ReplicatedMapEntryView} via the given {@link ReplicatedMapMergePolicy}. - * - * @param entryView the {@link ReplicatedMapEntryView} instance to merge - * @param mergePolicy the {@link ReplicatedMapMergePolicy} instance to apply - * @return {@code true} if merge is applied, otherwise {@code false} - */ - boolean merge(Object key, ReplicatedMapEntryView entryView, ReplicatedMapMergePolicy mergePolicy); } diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/HigherHitsMapMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/HigherHitsMapMergePolicy.java deleted file mode 100644 index 823949763ac13..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/HigherHitsMapMergePolicy.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.replicatedmap.impl.operation.ReplicatedMapDataSerializerHook; -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; - -/** - * Merges replicated map entries from source to destination map if the source entry - * has more hits than the destination one. - */ -public final class HigherHitsMapMergePolicy implements ReplicatedMapMergePolicy, IdentifiedDataSerializable { - - /** - * Single instance of this class. - */ - public static final HigherHitsMapMergePolicy INSTANCE = new HigherHitsMapMergePolicy(); - - private HigherHitsMapMergePolicy() { - } - - @Override - public Object merge(String mapName, ReplicatedMapEntryView mergingEntry, ReplicatedMapEntryView existingEntry) { - if (mergingEntry.getHits() >= existingEntry.getHits()) { - return mergingEntry.getValue(); - } - return existingEntry.getValue(); - } - - @Override - public int getFactoryId() { - return ReplicatedMapDataSerializerHook.F_ID; - } - - @Override - public int getClassId() { - return ReplicatedMapDataSerializerHook.HIGHER_HITS_MERGE_POLICY; - } - - @Override - public void writeData(ObjectDataOutput out) { - } - - @Override - public void readData(ObjectDataInput in) { - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/LatestUpdateMapMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/LatestUpdateMapMergePolicy.java deleted file mode 100644 index 0574b63622b80..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/LatestUpdateMapMergePolicy.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.replicatedmap.impl.operation.ReplicatedMapDataSerializerHook; -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; - -/** - * Merges replicated map entries from source to destination map if the source entry - * was updated more recently than the destination entry. - *

- * Note: This policy can only be used if the clocks of the nodes are in sync. - */ -public final class LatestUpdateMapMergePolicy implements ReplicatedMapMergePolicy, IdentifiedDataSerializable { - - /** - * Single instance of this class. - */ - public static final LatestUpdateMapMergePolicy INSTANCE = new LatestUpdateMapMergePolicy(); - - private LatestUpdateMapMergePolicy() { - } - - @Override - public Object merge(String mapName, ReplicatedMapEntryView mergingEntry, ReplicatedMapEntryView existingEntry) { - if (mergingEntry.getLastUpdateTime() >= existingEntry.getLastUpdateTime()) { - return mergingEntry.getValue(); - } - return existingEntry.getValue(); - } - - @Override - public int getFactoryId() { - return ReplicatedMapDataSerializerHook.F_ID; - } - - @Override - public int getClassId() { - return ReplicatedMapDataSerializerHook.LATEST_UPDATE_MERGE_POLICY; - } - - @Override - public void writeData(ObjectDataOutput out) { - } - - @Override - public void readData(ObjectDataInput in) { - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/MergePolicyProvider.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/MergePolicyProvider.java deleted file mode 100644 index 4716151682211..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/MergePolicyProvider.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.config.InvalidConfigurationException; -import com.hazelcast.spi.NodeEngine; -import com.hazelcast.spi.merge.SplitBrainMergePolicy; -import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; -import com.hazelcast.util.ConstructorFunction; - -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import static com.hazelcast.nio.ClassLoaderUtil.newInstance; -import static com.hazelcast.util.ConcurrencyUtil.getOrPutIfAbsent; - -/** - * A provider for {@link ReplicatedMapMergePolicy} instances. - */ -public final class MergePolicyProvider { - - private final ConcurrentMap mergePolicyMap - = new ConcurrentHashMap(); - - private final ConstructorFunction policyConstructorFunction - = new ConstructorFunction() { - @Override - public ReplicatedMapMergePolicy createNew(String className) { - try { - return newInstance(nodeEngine.getConfigClassLoader(), className); - } catch (Exception e) { - throw new InvalidConfigurationException("Invalid ReplicatedMapMergePolicy: " + className, e); - } - } - }; - - private final NodeEngine nodeEngine; - private final SplitBrainMergePolicyProvider policyProvider; - - public MergePolicyProvider(NodeEngine nodeEngine) { - this.nodeEngine = nodeEngine; - this.policyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); - addOutOfBoxPolicies(); - } - - private void addOutOfBoxPolicies() { - mergePolicyMap.put(PutIfAbsentMapMergePolicy.class.getName(), PutIfAbsentMapMergePolicy.INSTANCE); - mergePolicyMap.put(HigherHitsMapMergePolicy.class.getName(), HigherHitsMapMergePolicy.INSTANCE); - mergePolicyMap.put(PassThroughMergePolicy.class.getName(), PassThroughMergePolicy.INSTANCE); - mergePolicyMap.put(LatestUpdateMapMergePolicy.class.getName(), LatestUpdateMapMergePolicy.INSTANCE); - } - - /** - * Returns an instance of a merge policy by its classname. - *

- * First tries to resolve the classname as {@link SplitBrainMergePolicy}, - * then as {@link ReplicatedMapMergePolicy}. - *

- * If no merge policy matches an exception is thrown. - * - * @param className the classname of the given merge policy - * @return an instance of the merge policy class - */ - public Object getMergePolicy(String className) { - if (className == null) { - throw new InvalidConfigurationException("Class name is mandatory!"); - } - try { - return policyProvider.getMergePolicy(className); - } catch (InvalidConfigurationException e) { - return getOrPutIfAbsent(mergePolicyMap, className, policyConstructorFunction); - } - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/PassThroughMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/PassThroughMergePolicy.java deleted file mode 100644 index 4c04cdf446c52..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/PassThroughMergePolicy.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.replicatedmap.impl.operation.ReplicatedMapDataSerializerHook; -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; - -/** - * Merges replicated map entries from source to destination directly unless the merging entry is {@code null}. - */ -public final class PassThroughMergePolicy implements ReplicatedMapMergePolicy, IdentifiedDataSerializable { - - /** - * Single instance of this class. - */ - public static final PassThroughMergePolicy INSTANCE = new PassThroughMergePolicy(); - - private PassThroughMergePolicy() { - } - - @Override - public Object merge(String mapName, ReplicatedMapEntryView mergingEntry, ReplicatedMapEntryView existingEntry) { - return mergingEntry == null ? existingEntry.getValue() : mergingEntry.getValue(); - } - - @Override - public int getFactoryId() { - return ReplicatedMapDataSerializerHook.F_ID; - } - - @Override - public int getClassId() { - return ReplicatedMapDataSerializerHook.PASS_THROUGH_MERGE_POLICY; - } - - @Override - public void writeData(ObjectDataOutput out) { - } - - @Override - public void readData(ObjectDataInput in) { - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/PutIfAbsentMapMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/PutIfAbsentMapMergePolicy.java deleted file mode 100644 index e1bf45fa2139e..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/PutIfAbsentMapMergePolicy.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.replicatedmap.impl.operation.ReplicatedMapDataSerializerHook; -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; - -/** - * Merges replicated map entries from source to destination if they don't exist in the destination map. - */ -public final class PutIfAbsentMapMergePolicy implements ReplicatedMapMergePolicy, IdentifiedDataSerializable { - - /** - * Single instance of this class. - */ - public static final PutIfAbsentMapMergePolicy INSTANCE = new PutIfAbsentMapMergePolicy(); - - private PutIfAbsentMapMergePolicy() { - } - - @Override - public Object merge(String mapName, ReplicatedMapEntryView mergingEntry, ReplicatedMapEntryView existingEntry) { - if (existingEntry.getValue() == null) { - return mergingEntry.getValue(); - } - return existingEntry.getValue(); - } - - @Override - public int getFactoryId() { - return ReplicatedMapDataSerializerHook.F_ID; - } - - @Override - public int getClassId() { - return ReplicatedMapDataSerializerHook.PUT_IF_ABSENT_MERGE_POLICY; - } - - @Override - public void writeData(ObjectDataOutput out) { - } - - @Override - public void readData(ObjectDataInput in) { - } -} diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/ReplicatedMapMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/ReplicatedMapMergePolicy.java deleted file mode 100644 index 89ba0909fb073..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/ReplicatedMapMergePolicy.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; - -import java.io.Serializable; - -/** - * Policy for merging replicated map entries after a split-brain has been healed. - * - * @see com.hazelcast.replicatedmap.merge.HigherHitsMapMergePolicy - * @see com.hazelcast.replicatedmap.merge.LatestUpdateMapMergePolicy - * @see com.hazelcast.replicatedmap.merge.PassThroughMergePolicy - * @see com.hazelcast.replicatedmap.merge.PutIfAbsentMapMergePolicy - */ -public interface ReplicatedMapMergePolicy extends Serializable { - - /** - * Selects one of the merging and existing map entries to be merged. - *

- * Note that the {@code existingEntry} may be {@code null} if there - * is no entry with the same key in the destination map. - * This happens, when the entry for that key was - *

    - *
  • only created in the smaller sub-cluster during the split-brain
  • - *
  • removed in the larger sub-cluster during the split-brain
  • - *
- * - * @param mapName name of the replicated map - * @param mergingEntry {@link ReplicatedMapEntryView} instance that has the map entry to be merged - * @param existingEntry {@link ReplicatedMapEntryView} instance that has the existing map entry - * or {@code null} if there is no existing map entry - * @return the selected value for merging or {@code null} if the entry should be removed - */ - Object merge(String mapName, ReplicatedMapEntryView mergingEntry, ReplicatedMapEntryView existingEntry); -} diff --git a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/package-info.java b/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/package-info.java deleted file mode 100644 index 5bcf2d0fbe7c7..0000000000000 --- a/hazelcast/src/main/java/com/hazelcast/replicatedmap/merge/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Contains out-of-the-box merge policies for {@link com.hazelcast.replicatedmap.ReplicatedMap}. - */ -package com.hazelcast.replicatedmap.merge; diff --git a/hazelcast/src/main/java/com/hazelcast/spi/impl/merge/AbstractMergeRunnable.java b/hazelcast/src/main/java/com/hazelcast/spi/impl/merge/AbstractMergeRunnable.java index 4e1e4940954a6..32a1e1e884f43 100644 --- a/hazelcast/src/main/java/com/hazelcast/spi/impl/merge/AbstractMergeRunnable.java +++ b/hazelcast/src/main/java/com/hazelcast/spi/impl/merge/AbstractMergeRunnable.java @@ -18,7 +18,6 @@ import com.hazelcast.config.InMemoryFormat; import com.hazelcast.core.ExecutionCallback; -import com.hazelcast.internal.cluster.ClusterService; import com.hazelcast.internal.serialization.InternalSerializationService; import com.hazelcast.logging.ILogger; import com.hazelcast.map.IMap; @@ -27,12 +26,13 @@ import com.hazelcast.nio.serialization.DataType; import com.hazelcast.replicatedmap.ReplicatedMap; import com.hazelcast.spi.NodeEngine; +import com.hazelcast.spi.SplitBrainHandlerService; import com.hazelcast.spi.impl.operationservice.Operation; import com.hazelcast.spi.impl.operationservice.OperationFactory; import com.hazelcast.spi.impl.operationservice.OperationService; -import com.hazelcast.spi.SplitBrainHandlerService; import com.hazelcast.spi.merge.MergingEntry; import com.hazelcast.spi.merge.SplitBrainMergePolicy; +import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; import com.hazelcast.spi.partition.IPartitionService; import com.hazelcast.util.MutableLong; @@ -47,7 +47,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; -import static com.hazelcast.internal.config.MergePolicyValidator.checkMergePolicySupportsInMemoryFormat; import static com.hazelcast.util.ExceptionUtil.rethrow; import static java.util.Collections.singleton; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -67,15 +66,15 @@ public abstract class AbstractMergeRunnable splitBrainHandlerService; private final InternalSerializationService serializationService; + private final AbstractSplitBrainHandlerService splitBrainHandlerService; + private final Semaphore semaphore = new Semaphore(0); private Map> mergingStoresByName; @@ -87,7 +86,7 @@ protected AbstractMergeRunnable(String serviceName, this.serviceName = serviceName; this.logger = nodeEngine.getLogger(getClass()); this.partitionService = nodeEngine.getPartitionService(); - this.clusterService = nodeEngine.getClusterService(); + this.mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); this.operationService = nodeEngine.getOperationService(); this.serializationService = (InternalSerializationService) nodeEngine.getSerializationService(); this.splitBrainHandlerService = splitBrainHandlerService; @@ -98,11 +97,8 @@ private Map> groupStoresByName(Collection store for (Store store : stores) { String dataStructureName = getDataStructureName(store); - Collection storeList = storesByName.get(dataStructureName); - if (storeList == null) { - storeList = new LinkedList(); - storesByName.put(dataStructureName, storeList); - } + Collection storeList + = storesByName.computeIfAbsent(dataStructureName, k -> new LinkedList<>()); storeList.add(store); } return storesByName; @@ -114,7 +110,6 @@ public final void run() { int mergedCount = 0; mergedCount += mergeWithSplitBrainMergePolicy(); - mergedCount += mergeWithLegacyMergePolicy(); waitMergeEnd(mergedCount); } @@ -132,78 +127,28 @@ private int mergeWithSplitBrainMergePolicy() { String dataStructureName = entry.getKey(); Collection stores = entry.getValue(); - if (getMergePolicy(dataStructureName) instanceof SplitBrainMergePolicy) { - MergingItemBiConsumer consumer = newConsumer(dataStructureName); - for (Store store : stores) { - try { - mergeStore(store, consumer); - consumer.consumeRemaining(); - } finally { - asyncDestroyStores(singleton(store)); - } + MergingItemBiConsumer consumer = newConsumer(dataStructureName); + for (Store store : stores) { + try { + mergeStore(store, consumer); + consumer.consumeRemaining(); + } finally { + asyncDestroyStores(singleton(store)); } - mergedCount += consumer.mergedCount; - onMerge(dataStructureName); - iterator.remove(); } + mergedCount += consumer.mergedCount; + onMerge(dataStructureName); + iterator.remove(); } return mergedCount; } private MergingItemBiConsumer newConsumer(String dataStructureName) { - SplitBrainMergePolicy policy = getSplitBrainMergePolicy(dataStructureName); + SplitBrainMergePolicy policy = getMergePolicy(dataStructureName); int batchSize = getBatchSize(dataStructureName); return new MergingItemBiConsumer(dataStructureName, policy, batchSize); } - @SuppressWarnings("unchecked") - private SplitBrainMergePolicy getSplitBrainMergePolicy(String dataStructureName) { - return ((SplitBrainMergePolicy) getMergePolicy(dataStructureName)); - } - - private int mergeWithLegacyMergePolicy() { - LegacyOperationBiConsumer consumer = new LegacyOperationBiConsumer(); - - Iterator>> iterator = mergingStoresByName.entrySet().iterator(); - while (iterator.hasNext()) { - try { - Map.Entry> entry = iterator.next(); - - String dataStructureName = entry.getKey(); - Collection stores = entry.getValue(); - - if (canMergeLegacy(dataStructureName)) { - for (Store store : stores) { - try { - mergeStoreLegacy(store, consumer); - } finally { - asyncDestroyStores(singleton(store)); - } - } - onMerge(dataStructureName); - } else { - asyncDestroyStores(stores); - } - } finally { - iterator.remove(); - } - } - - return consumer.mergedCount; - } - - /** - * Check if data structures in-memory-format appropriate to merge - * with legacy policies - */ - private boolean canMergeLegacy(String dataStructureName) { - Object mergePolicy = getMergePolicy(dataStructureName); - InMemoryFormat inMemoryFormat = getInMemoryFormat(dataStructureName); - - return checkMergePolicySupportsInMemoryFormat(dataStructureName, - mergePolicy, inMemoryFormat, false, logger); - } - private void waitMergeEnd(int mergedCount) { try { long timeoutMillis = Math.max(mergedCount * TIMEOUT_FACTOR, MINIMAL_TIMEOUT_MILLIS); @@ -266,7 +211,7 @@ private void init() { public void accept(Integer partitionId, MergingItem mergingItem) { List entries = mergingItemsPerPartition[partitionId]; if (entries == null) { - entries = new LinkedList(); + entries = new LinkedList<>(); mergingItemsPerPartition[partitionId] = entries; } @@ -396,11 +341,6 @@ protected void onMerge(String dataStructureName) { */ protected abstract void mergeStore(Store recordStore, BiConsumer consumer); - /** - * Used to merge with legacy merge policies. - */ - protected abstract void mergeStoreLegacy(Store recordStore, BiConsumer consumer); - /** * This batch size can only be used with {@link SplitBrainMergePolicy}, * legacy merge policies don't support batch data sending. @@ -410,9 +350,9 @@ protected void onMerge(String dataStructureName) { protected abstract int getBatchSize(String dataStructureName); /** - * @return a type of {@link SplitBrainMergePolicy} or a legacy merge policy + * @return a type of {@link SplitBrainMergePolicy} */ - protected abstract Object getMergePolicy(String dataStructureName); + protected abstract SplitBrainMergePolicy getMergePolicy(String dataStructureName); protected abstract String getDataStructureName(Store store); diff --git a/hazelcast/src/main/java/com/hazelcast/spi/merge/SplitBrainMergePolicy.java b/hazelcast/src/main/java/com/hazelcast/spi/merge/SplitBrainMergePolicy.java index 1487ec6968852..fc6bbf0d34df4 100644 --- a/hazelcast/src/main/java/com/hazelcast/spi/merge/SplitBrainMergePolicy.java +++ b/hazelcast/src/main/java/com/hazelcast/spi/merge/SplitBrainMergePolicy.java @@ -23,44 +23,59 @@ import com.hazelcast.spi.NodeAware; /** - * Policy for merging data structure values after a split-brain has been healed. + * Policy for merging data structure values + * after a split-brain has been healed. *

- * The values of merging and existing {@link MergingValue}s are always in the in-memory format of the - * backing data structure. This can be a serialized format, so the content cannot be processed without deserialization. - * For most merge policies this will be fine, since the key or value are not used. + * The values of merging and existing {@link MergingValue}s are + * always in the in-memory format of the backing data structure. + * This can be a serialized format, so the content cannot be + * processed without deserialization. For most merge policies + * this will be fine, since the key or value are not used. *

* The deserialization is not done eagerly for two main reasons: *

    - *
  • The deserialization is quite expensive and should be avoided, if the result is not needed.
  • - *
  • There is no need to locate classes of stored entries on the server side, when the entries are not deserialized. - * So you can put entries from a client by using {@link com.hazelcast.config.InMemoryFormat#BINARY} with a different - * classpath on client and server. In this case a deserialization could throw a {@link java.lang.ClassNotFoundException}.
  • + *
  • The deserialization is quite expensive and + * should be avoided, if the result is not needed.
  • + *
  • There is no need to locate classes of stored entries + * on the server side, when the entries are not deserialized. + * So you can put entries from a client by using {@link + * com.hazelcast.config.InMemoryFormat#BINARY} with a different + * classpath on client and server. In this case a deserialization + * could throw a {@link java.lang.ClassNotFoundException}.
  • *
- * If you need the deserialized data you can call {@link MergingValue#getDeserializedValue()} or - * {@link MergingEntry#getDeserializedKey()}, which will deserialize the data lazily. + * If you need the deserialized data you can call + * {@link MergingValue#getDeserializedValue()} + * or {@link MergingEntry#getDeserializedKey()}, + * which will deserialize the data lazily. *

- * A merge policy can implement {@link HazelcastInstanceAware} to get the {@link HazelcastInstance} injected. - * This can be used to retrieve the user context via {@link HazelcastInstance#getUserContext()}, - * which is an easy way to get user dependencies that are otherwise hard to obtain. + * A merge policy can implement {@link HazelcastInstanceAware} to get the + * {@link HazelcastInstance} injected. This can be used to retrieve the + * user context via {@link HazelcastInstance#getUserContext()}, which is + * an easy way to get user dependencies that are otherwise hard to obtain. *

- * A merge policy can also implement {@link NodeAware} to get an instance of {@link Node} injected. + * A merge policy can also implement {@link NodeAware} + * to get an instance of {@link Node} injected. * * @param the type of the returned merged value - * @param the type of the required merging value, e.g. a simple {@code MergingValue} - * or a composition like {@code MergingEntry & MergingHits & MergingLastAccessTime} + * @param the type of the required merging value, e.g. a simple + * {@code MergingValue} or a composition like {@code + * MergingEntry & MergingHits & MergingLastAccessTime} * @since 3.10 */ -public interface SplitBrainMergePolicy> extends DataSerializable { +public interface SplitBrainMergePolicy> + extends DataSerializable { /** - * Selects the value of either the merging or the existing {@link MergingValue} which should be merged. + * Selects the value of either the merging or the + * existing {@link MergingValue} which should be merged. *

* Note that the existing {@link MergingValue} instance may be {@code null} * if no matching data could be found to the merging {@link MergingValue}. * - * @param mergingValue {@link MergingValue} instance that has the merging data of the smaller sub-cluster - * @param existingValue {@link MergingValue} instance that has the existing data - * or {@code null} if no matching data exists + * @param mergingValue {@link MergingValue} instance that has the + * merging data of the smaller sub-cluster + * @param existingValue {@link MergingValue} instance that has the existing + * data or {@code null} if no matching data exists * @return the selected value for merging */ V merge(T mergingValue, T existingValue); diff --git a/hazelcast/src/main/java/com/hazelcast/spi/merge/SplitBrainMergePolicyProvider.java b/hazelcast/src/main/java/com/hazelcast/spi/merge/SplitBrainMergePolicyProvider.java index eea513491d395..a6f7826f3694e 100644 --- a/hazelcast/src/main/java/com/hazelcast/spi/merge/SplitBrainMergePolicyProvider.java +++ b/hazelcast/src/main/java/com/hazelcast/spi/merge/SplitBrainMergePolicyProvider.java @@ -40,7 +40,7 @@ public final class SplitBrainMergePolicyProvider { private static final Map OUT_OF_THE_BOX_MERGE_POLICIES; static { - OUT_OF_THE_BOX_MERGE_POLICIES = new HashMap(); + OUT_OF_THE_BOX_MERGE_POLICIES = new HashMap<>(); addPolicy(DiscardMergePolicy.class, new DiscardMergePolicy()); addPolicy(ExpirationTimeMergePolicy.class, new ExpirationTimeMergePolicy()); addPolicy(HigherHitsMergePolicy.class, new HigherHitsMergePolicy()); diff --git a/hazelcast/src/main/resources/hazelcast-config-4.0.xsd b/hazelcast/src/main/resources/hazelcast-config-4.0.xsd index 3bcedea9b2bc9..cc11eede8419e 100644 --- a/hazelcast/src/main/resources/hazelcast-config-4.0.xsd +++ b/hazelcast/src/main/resources/hazelcast-config-4.0.xsd @@ -3394,28 +3394,6 @@ Resolve conflicts that occurred when target cluster already has the replicated entry key. - - 4 merge policy implementations for IMap and 2 merge policy implementations for - ICache are provided out-of-the-box. - - IMap has the following merge policies: - com.hazelcast.map.merge.PutIfAbsentMapMergePolicy: Incoming entry merges from the - source map to the target map if it does not exist in the target map. - com.hazelcast.map.merge.HigherHitsMapMergePolicy: Incoming entry merges from the - source map to the target map if the source entry has more hits than the target one. - com.hazelcast.map.merge.PassThroughMergePolicy: Incoming entry merges from the - source map to the target map unless the incoming entry is not null. - com.hazelcast.map.merge.LatestUpdateMapMergePolicy: Incoming entry merges from the - source map to the target map if the source entry has been updated more recently - than the target entry. Please note that this merge policy can only be used when the - clusters' clocks are in sync. - - ICache has the following merge policies: - com.hazelcast.cache.merge.HigherHitsCacheMergePolicy: Incoming entry merges from - the source cache to the target cache if the source entry has more hits than the - target one. - com.hazelcast.cache.merge.PassThroughCacheMergePolicy: Incoming entry merges from - the source cache to the target cache unless the incoming entry is not null. diff --git a/hazelcast/src/test/java/com/hazelcast/cache/merge/AbstractCacheMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/cache/merge/AbstractCacheMergePolicyTest.java deleted file mode 100644 index 8d2bd9eead4fe..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/cache/merge/AbstractCacheMergePolicyTest.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.CacheMergePolicy; -import org.junit.Before; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public abstract class AbstractCacheMergePolicyTest { - - private static final String EXISTING = "EXISTING"; - private static final String MERGING = "MERGING"; - - protected CacheMergePolicy policy; - - @Before - public void setup() { - policy = createCacheMergePolicy(); - } - - protected abstract CacheMergePolicy createCacheMergePolicy(); - - @Test - public void merge_mergingWins() { - CacheEntryView existing = entryWithGivenPropertyAndValue(1, EXISTING); - CacheEntryView merging = entryWithGivenPropertyAndValue(333, MERGING); - - assertEquals(MERGING, policy.merge("cache", merging, existing)); - } - - @Test - @SuppressWarnings("ConstantConditions") - public void merge_mergingWins_sinceExistingIsNotExist() { - CacheEntryView existing = null; - CacheEntryView merging = entryWithGivenPropertyAndValue(1, MERGING); - - assertEquals(MERGING, policy.merge("cache", merging, existing)); - } - - @Test - public void merge_existingWins() { - CacheEntryView existing = entryWithGivenPropertyAndValue(333, EXISTING); - CacheEntryView merging = entryWithGivenPropertyAndValue(1, MERGING); - - assertEquals(EXISTING, policy.merge("cache", merging, existing)); - } - - @Test - public void merge_draw_mergingWins() { - CacheEntryView existing = entryWithGivenPropertyAndValue(1, EXISTING); - CacheEntryView merging = entryWithGivenPropertyAndValue(1, MERGING); - - assertEquals(MERGING, policy.merge("cache", merging, existing)); - } - - private CacheEntryView entryWithGivenPropertyAndValue(long testedProperty, String value) { - CacheEntryView entryView = mock(CacheEntryView.class); - try { - if (policy instanceof HigherHitsCacheMergePolicy) { - when(entryView.getAccessHit()).thenReturn(testedProperty); - } else if (policy instanceof LatestAccessCacheMergePolicy) { - when(entryView.getLastAccessTime()).thenReturn(testedProperty); - } - when(entryView.getValue()).thenReturn(value); - return entryView; - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/cache/merge/CacheMergePolicyProviderTest.java b/hazelcast/src/test/java/com/hazelcast/cache/merge/CacheMergePolicyProviderTest.java deleted file mode 100644 index 6be3d5273634a..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/cache/merge/CacheMergePolicyProviderTest.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.BuiltInCacheMergePolicies; -import com.hazelcast.cache.CacheMergePolicy; -import com.hazelcast.cache.impl.CacheService; -import com.hazelcast.cache.impl.merge.policy.CacheMergePolicyProvider; -import com.hazelcast.config.InvalidConfigurationException; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.HazelcastTestSupport; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.hamcrest.core.IsInstanceOf; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class CacheMergePolicyProviderTest extends HazelcastTestSupport { - - private CacheMergePolicyProvider mergePolicyProvider; - - @Rule - public ExpectedException expected = ExpectedException.none(); - - @Before - public void setup() { - CacheService service = getNodeEngineImpl(createHazelcastInstance()).getService(CacheService.SERVICE_NAME); - mergePolicyProvider = service.getMergePolicyProvider(); - } - - @Test - public void getMergePolicy_NotExistingMergePolicy() { - expected.expect(InvalidConfigurationException.class); - expected.expectCause(IsInstanceOf.any(ClassNotFoundException.class)); - mergePolicyProvider.getMergePolicy("No such policy!"); - } - - @Test - public void getMergePolicy_NullPolicy() { - expected.expect(InvalidConfigurationException.class); - mergePolicyProvider.getMergePolicy(null); - } - - @Test - public void getMergePolicy_withClassName_PutIfAbsentCacheMergePolicy() { - assertMergePolicyCorrectlyInitialised(PutIfAbsentCacheMergePolicy.class.getName(), PutIfAbsentCacheMergePolicy.class); - } - - @Test - public void getMergePolicy_withConstant_PutIfAbsentCacheMergePolicy() { - assertMergePolicyCorrectlyInitialised(BuiltInCacheMergePolicies.PUT_IF_ABSENT.name(), PutIfAbsentCacheMergePolicy.class); - } - - @Test - public void getMergePolicy_withClassName_LatestAccessCacheMergePolicy() { - assertMergePolicyCorrectlyInitialised(LatestAccessCacheMergePolicy.class.getName(), LatestAccessCacheMergePolicy.class); - } - - @Test - public void getMergePolicy_withConstant_LatestAccessCacheMergePolicy() { - assertMergePolicyCorrectlyInitialised(BuiltInCacheMergePolicies.LATEST_ACCESS.name(), LatestAccessCacheMergePolicy.class); - } - - @Test - public void getMergePolicy_withClassName_PassThroughCachePolicy() { - assertMergePolicyCorrectlyInitialised(PassThroughCacheMergePolicy.class.getName(), PassThroughCacheMergePolicy.class); - } - - @Test - public void getMergePolicy_withConstant_PassThroughCachePolicy() { - assertMergePolicyCorrectlyInitialised(BuiltInCacheMergePolicies.PASS_THROUGH.name(), PassThroughCacheMergePolicy.class); - } - - @Test - public void getMergePolicy_withClassName_HigherHitsMapCachePolicy() { - assertMergePolicyCorrectlyInitialised(HigherHitsCacheMergePolicy.class.getName(), HigherHitsCacheMergePolicy.class); - } - - @Test - public void getMergePolicy_withConstant_HigherHitsMapCachePolicy() { - assertMergePolicyCorrectlyInitialised(BuiltInCacheMergePolicies.HIGHER_HITS.name(), HigherHitsCacheMergePolicy.class); - } - - private void assertMergePolicyCorrectlyInitialised(String mergePolicyName, - Class expectedMergePolicyClass) { - Object mergePolicy = mergePolicyProvider.getMergePolicy(mergePolicyName); - - assertNotNull(mergePolicy); - assertEquals(expectedMergePolicyClass, mergePolicy.getClass()); - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/cache/merge/HigherHitsCacheMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/cache/merge/HigherHitsCacheMergePolicyTest.java deleted file mode 100644 index cdfbd10320264..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/cache/merge/HigherHitsCacheMergePolicyTest.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheMergePolicy; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class HigherHitsCacheMergePolicyTest extends AbstractCacheMergePolicyTest { - - @Override - protected CacheMergePolicy createCacheMergePolicy() { - return new HigherHitsCacheMergePolicy(); - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/cache/merge/LatestAccessCacheMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/cache/merge/LatestAccessCacheMergePolicyTest.java deleted file mode 100644 index 7ffd8cfa4b6dc..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/cache/merge/LatestAccessCacheMergePolicyTest.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheMergePolicy; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class LatestAccessCacheMergePolicyTest extends AbstractCacheMergePolicyTest { - - @Override - protected CacheMergePolicy createCacheMergePolicy() { - return new LatestAccessCacheMergePolicy(); - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/cache/merge/LegacyCacheSplitBrainTest.java b/hazelcast/src/test/java/com/hazelcast/cache/merge/LegacyCacheSplitBrainTest.java deleted file mode 100644 index 7dc0ea69c48ff..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/cache/merge/LegacyCacheSplitBrainTest.java +++ /dev/null @@ -1,243 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.CacheMergePolicy; -import com.hazelcast.cache.impl.HazelcastServerCachingProvider; -import com.hazelcast.config.CacheConfig; -import com.hazelcast.config.InMemoryFormat; -import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.instance.impl.HazelcastInstanceImpl; -import com.hazelcast.test.HazelcastParallelParametersRunnerFactory; -import com.hazelcast.test.SplitBrainTestSupport; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; -import org.junit.runners.Parameterized.UseParametersRunnerFactory; - -import javax.cache.Cache; -import javax.cache.CacheManager; -import javax.cache.spi.CachingProvider; -import java.util.Collection; - -import static com.hazelcast.config.InMemoryFormat.BINARY; -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -@RunWith(Parameterized.class) -@UseParametersRunnerFactory(HazelcastParallelParametersRunnerFactory.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -@SuppressWarnings("unchecked") -public class LegacyCacheSplitBrainTest extends SplitBrainTestSupport { - - @Parameters(name = "inMemoryFormat:{0}, mergePolicy:{1}") - public static Collection parameters() { - return asList(new Object[][]{ - {BINARY, LatestAccessCacheMergePolicy.class}, - {BINARY, HigherHitsCacheMergePolicy.class}, - {BINARY, PutIfAbsentCacheMergePolicy.class}, - {BINARY, PassThroughCacheMergePolicy.class}, - {BINARY, CustomCacheMergePolicy.class}, - }); - } - - @Parameter - public InMemoryFormat inMemoryFormat; - - @Parameter(value = 1) - public Class mergePolicyClass; - - private String cacheName = randomMapName(); - private Cache cache1; - private Cache cache2; - private MergeLifecycleListener mergeLifecycleListener; - - @Override - protected void onBeforeSplitBrainCreated(HazelcastInstance[] instances) { - warmUpPartitions(instances); - } - - @Override - protected void onAfterSplitBrainCreated(HazelcastInstance[] firstBrain, HazelcastInstance[] secondBrain) { - mergeLifecycleListener = new MergeLifecycleListener(secondBrain.length); - for (HazelcastInstance instance : secondBrain) { - instance.getLifecycleService().addLifecycleListener(mergeLifecycleListener); - } - - CacheConfig cacheConfig = newCacheConfig(cacheName, mergePolicyClass, inMemoryFormat); - cache1 = createCache(firstBrain[0], cacheConfig); - cache2 = createCache(secondBrain[0], cacheConfig); - - if (mergePolicyClass == LatestAccessCacheMergePolicy.class) { - afterSplitLatestAccessCacheMergePolicy(); - } else if (mergePolicyClass == HigherHitsCacheMergePolicy.class) { - afterSplitHigherHitsCacheMergePolicy(); - } else if (mergePolicyClass == PutIfAbsentCacheMergePolicy.class) { - afterSplitPutIfAbsentCacheMergePolicy(); - } else if (mergePolicyClass == PassThroughCacheMergePolicy.class) { - afterSplitPassThroughCacheMergePolicy(); - } else if (mergePolicyClass == CustomCacheMergePolicy.class) { - afterSplitCustomCacheMergePolicy(); - } else { - fail(); - } - } - - @Override - protected void onAfterSplitBrainHealed(HazelcastInstance[] instances) { - // wait until merge completes - mergeLifecycleListener.await(); - - if (mergePolicyClass == LatestAccessCacheMergePolicy.class) { - afterMergeLatestAccessCacheMergePolicy(); - } else if (mergePolicyClass == HigherHitsCacheMergePolicy.class) { - afterMergeHigherHitsCacheMergePolicy(); - } else if (mergePolicyClass == PutIfAbsentCacheMergePolicy.class) { - afterMergePutIfAbsentCacheMergePolicy(); - } else if (mergePolicyClass == PassThroughCacheMergePolicy.class) { - afterMergePassThroughCacheMergePolicy(); - } else if (mergePolicyClass == CustomCacheMergePolicy.class) { - afterMergeCustomCacheMergePolicy(); - } else { - fail(); - } - } - - private void afterSplitLatestAccessCacheMergePolicy() { - cache1.put("key1", "value"); - // access to record - assertEquals("value", cache1.get("key1")); - - // prevent updating at the same time - sleepAtLeastMillis(100); - - cache2.put("key1", "LatestAccessedValue"); - // access to record - assertEquals("LatestAccessedValue", cache2.get("key1")); - - cache2.put("key2", "value2"); - // access to record - assertEquals("value2", cache2.get("key2")); - - // prevent updating at the same time - sleepAtLeastMillis(100); - - cache1.put("key2", "LatestAccessedValue2"); - // access to record - assertEquals("LatestAccessedValue2", cache1.get("key2")); - } - - private void afterMergeLatestAccessCacheMergePolicy() { - assertEquals("LatestAccessedValue", cache1.get("key1")); - assertEquals("LatestAccessedValue", cache2.get("key1")); - - assertEquals("LatestAccessedValue2", cache1.get("key2")); - assertEquals("LatestAccessedValue2", cache2.get("key2")); - } - - private void afterSplitHigherHitsCacheMergePolicy() { - cache1.put("key1", "higherHitsValue"); - cache1.put("key2", "value2"); - - // increase hits number - assertEquals("higherHitsValue", cache1.get("key1")); - assertEquals("higherHitsValue", cache1.get("key1")); - - cache2.put("key1", "value1"); - cache2.put("key2", "higherHitsValue2"); - - // increase hits number - assertEquals("higherHitsValue2", cache2.get("key2")); - assertEquals("higherHitsValue2", cache2.get("key2")); - } - - private void afterMergeHigherHitsCacheMergePolicy() { - assertEquals("higherHitsValue", cache1.get("key1")); - assertEquals("higherHitsValue", cache2.get("key1")); - - assertEquals("higherHitsValue2", cache1.get("key2")); - assertEquals("higherHitsValue2", cache2.get("key2")); - } - - private void afterSplitPutIfAbsentCacheMergePolicy() { - cache1.put("key1", "PutIfAbsentValue1"); - - cache2.put("key1", "value"); - cache2.put("key2", "PutIfAbsentValue2"); - } - - private void afterMergePutIfAbsentCacheMergePolicy() { - assertEquals("PutIfAbsentValue1", cache1.get("key1")); - assertEquals("PutIfAbsentValue1", cache2.get("key1")); - - assertEquals("PutIfAbsentValue2", cache1.get("key2")); - assertEquals("PutIfAbsentValue2", cache2.get("key2")); - } - - private void afterSplitPassThroughCacheMergePolicy() { - cache1.put("key", "value"); - cache2.put("key", "passThroughValue"); - } - - private void afterMergePassThroughCacheMergePolicy() { - assertEquals("passThroughValue", cache1.get("key")); - assertEquals("passThroughValue", cache2.get("key")); - } - - private void afterSplitCustomCacheMergePolicy() { - cache1.put("key", "value"); - cache2.put("key", 1); - } - - private void afterMergeCustomCacheMergePolicy() { - assertEquals(1, cache1.get("key")); - assertEquals(1, cache2.get("key")); - } - - private static Cache createCache(HazelcastInstance hz, CacheConfig cacheConfig) { - HazelcastInstanceImpl hazelcastInstanceImpl = getHazelcastInstanceImpl(hz); - CachingProvider cachingProvider = HazelcastServerCachingProvider.createCachingProvider(hazelcastInstanceImpl); - CacheManager cacheManager = cachingProvider.getCacheManager(); - return cacheManager.createCache(cacheConfig.getName(), cacheConfig); - } - - protected CacheConfig newCacheConfig(String cacheName, Class mergePolicy, - InMemoryFormat inMemoryFormat) { - CacheConfig cacheConfig = new CacheConfig(); - cacheConfig.setName(cacheName); - cacheConfig.setMergePolicy(mergePolicy.getName()); - cacheConfig.setInMemoryFormat(inMemoryFormat); - return cacheConfig; - } - - protected static class CustomCacheMergePolicy implements CacheMergePolicy { - - @Override - public Object merge(String cacheName, CacheEntryView mergingEntry, CacheEntryView existingEntry) { - if (mergingEntry.getValue() instanceof Integer) { - return mergingEntry.getValue(); - } - return null; - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/cache/merge/PassThroughCacheMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/cache/merge/PassThroughCacheMergePolicyTest.java deleted file mode 100644 index f26e397a98f29..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/cache/merge/PassThroughCacheMergePolicyTest.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.CacheMergePolicy; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class PassThroughCacheMergePolicyTest { - - private static final String EXISTING = "EXISTING"; - private static final String MERGING = "MERGING"; - - protected CacheMergePolicy policy; - - @Before - public void setup() { - policy = new PassThroughCacheMergePolicy(); - } - - @Test - public void merge_mergingNotNull() { - CacheEntryView existing = entryWithGivenValue(EXISTING); - CacheEntryView merging = entryWithGivenValue(MERGING); - - assertEquals(MERGING, policy.merge("cache", merging, existing)); - } - - @Test - @SuppressWarnings("ConstantConditions") - public void merge_mergingNull() { - CacheEntryView existing = entryWithGivenValue(EXISTING); - CacheEntryView merging = null; - - assertEquals(EXISTING, policy.merge("cache", merging, existing)); - } - - private CacheEntryView entryWithGivenValue(String value) { - CacheEntryView entryView = mock(CacheEntryView.class); - try { - when(entryView.getValue()).thenReturn(value); - return entryView; - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/cache/merge/PutIfAbsentCacheMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/cache/merge/PutIfAbsentCacheMergePolicyTest.java deleted file mode 100644 index fad13cc0de4eb..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/cache/merge/PutIfAbsentCacheMergePolicyTest.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.cache.merge; - -import com.hazelcast.cache.CacheEntryView; -import com.hazelcast.cache.CacheMergePolicy; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class PutIfAbsentCacheMergePolicyTest { - - private static final String EXISTING = "EXISTING"; - private static final String MERGING = "MERGING"; - - protected CacheMergePolicy policy; - - @Before - public void setup() { - policy = new PutIfAbsentCacheMergePolicy(); - } - - @Test - @SuppressWarnings("ConstantConditions") - public void merge_existingValueAbsent() { - CacheEntryView existing = null; - CacheEntryView merging = entryWithGivenValue(MERGING); - - assertEquals(MERGING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_existingValuePresent() { - CacheEntryView existing = entryWithGivenValue(EXISTING); - CacheEntryView merging = entryWithGivenValue(MERGING); - - assertEquals(EXISTING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_bothValuesNull() { - CacheEntryView existing = entryWithGivenValue(null); - CacheEntryView merging = entryWithGivenValue(null); - - assertNull(policy.merge("map", merging, existing)); - } - - private CacheEntryView entryWithGivenValue(String value) { - CacheEntryView entryView = mock(CacheEntryView.class); - try { - when(entryView.getValue()).thenReturn(value); - return entryView; - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/cluster/SplitBrainHandlerTest.java b/hazelcast/src/test/java/com/hazelcast/cluster/SplitBrainHandlerTest.java index 6bf3de366abe2..74e5d97da8d62 100644 --- a/hazelcast/src/test/java/com/hazelcast/cluster/SplitBrainHandlerTest.java +++ b/hazelcast/src/test/java/com/hazelcast/cluster/SplitBrainHandlerTest.java @@ -28,7 +28,7 @@ import com.hazelcast.instance.FirewallingNodeContext; import com.hazelcast.instance.impl.HazelcastInstanceFactory; import com.hazelcast.internal.util.RuntimeAvailableProcessors; -import com.hazelcast.map.merge.PassThroughMergePolicy; +import com.hazelcast.spi.merge.PassThroughMergePolicy; import com.hazelcast.spi.properties.GroupProperty; import com.hazelcast.test.AssertTask; import com.hazelcast.test.HazelcastSerialClassRunner; @@ -281,7 +281,7 @@ public void testTcpIpSplitBrainJoinsCorrectCluster() throws Exception { * h4 to restart it will have to be notified by h3. */ h3.getConfig().getNetworkConfig().getJoin().getTcpIpConfig().setMembers(allMembers); - h4.getConfig().getNetworkConfig().getJoin().getTcpIpConfig().clear().setMembers(Collections.emptyList()); + h4.getConfig().getNetworkConfig().getJoin().getTcpIpConfig().clear().setMembers(Collections.emptyList()); assertTrue(latch.await(60, TimeUnit.SECONDS)); @@ -607,7 +607,9 @@ private Config buildConfig(final String groupName, final boolean liteMember) { JoinConfig join = networkConfig.getJoin(); join.getMulticastConfig().setEnabled(true); - config.getMapConfig("default").setMergePolicy(PassThroughMergePolicy.class.getName()); + config.getMapConfig("default") + .getMergePolicyConfig() + .setPolicy(PassThroughMergePolicy.class.getName()); return config; } diff --git a/hazelcast/src/test/java/com/hazelcast/config/MapConfigReadOnlyTest.java b/hazelcast/src/test/java/com/hazelcast/config/MapConfigReadOnlyTest.java index ed8043f7a6f50..d4e2aabe31d92 100644 --- a/hazelcast/src/test/java/com/hazelcast/config/MapConfigReadOnlyTest.java +++ b/hazelcast/src/test/java/com/hazelcast/config/MapConfigReadOnlyTest.java @@ -235,11 +235,6 @@ public void setNearCacheConfigOfReadOnlyMapConfigShouldFail() { getReadOnlyConfig().setNearCacheConfig(new NearCacheConfig()); } - @Test(expected = UnsupportedOperationException.class) - public void setMergePolicyOfReadOnlyMapConfigShouldFail() { - getReadOnlyConfig().setMergePolicy("myMergePolicy"); - } - @Test(expected = UnsupportedOperationException.class) public void setStatisticsEnabledOfReadOnlyMapConfigShouldFail() { getReadOnlyConfig().setStatisticsEnabled(true); diff --git a/hazelcast/src/test/java/com/hazelcast/config/ReplicatedMapConfigReadOnlyTest.java b/hazelcast/src/test/java/com/hazelcast/config/ReplicatedMapConfigReadOnlyTest.java index 29f7cc036a488..5ae61e504025e 100644 --- a/hazelcast/src/test/java/com/hazelcast/config/ReplicatedMapConfigReadOnlyTest.java +++ b/hazelcast/src/test/java/com/hazelcast/config/ReplicatedMapConfigReadOnlyTest.java @@ -81,11 +81,6 @@ public void testSetQuorumName() { getReadOnlyConfig().setQuorumName("myQuorum"); } - @Test(expected = UnsupportedOperationException.class) - public void testSetMergePolicy() { - getReadOnlyConfig().setMergePolicy("MyMergePolicy"); - } - @Test(expected = UnsupportedOperationException.class) public void testSetMergePolicyConfig() { getReadOnlyConfig().setMergePolicyConfig(new MergePolicyConfig()); diff --git a/hazelcast/src/test/java/com/hazelcast/internal/config/ConfigValidatorTest.java b/hazelcast/src/test/java/com/hazelcast/internal/config/ConfigValidatorTest.java index de3e535c86ac2..092ca16569b06 100644 --- a/hazelcast/src/test/java/com/hazelcast/internal/config/ConfigValidatorTest.java +++ b/hazelcast/src/test/java/com/hazelcast/internal/config/ConfigValidatorTest.java @@ -16,7 +16,6 @@ package com.hazelcast.internal.config; -import com.hazelcast.cache.impl.merge.policy.CacheMergePolicyProvider; import com.hazelcast.config.CacheSimpleConfig; import com.hazelcast.config.Config; import com.hazelcast.config.EvictionConfig; @@ -24,7 +23,6 @@ import com.hazelcast.config.MapConfig; import com.hazelcast.config.NativeMemoryConfig; import com.hazelcast.config.cp.CPSubsystemConfig; -import com.hazelcast.map.merge.MergePolicyProvider; import com.hazelcast.spi.NodeEngine; import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; import com.hazelcast.spi.properties.HazelcastProperties; @@ -53,8 +51,7 @@ public class ConfigValidatorTest extends HazelcastTestSupport { private HazelcastProperties properties; private NativeMemoryConfig nativeMemoryConfig; - private MergePolicyProvider mapMergePolicyProvider; - private CacheMergePolicyProvider cacheMergePolicyProvider; + private SplitBrainMergePolicyProvider splitBrainMergePolicyProvider; @Before public void setUp() { @@ -63,11 +60,9 @@ public void setUp() { NodeEngine nodeEngine = Mockito.mock(NodeEngine.class); when(nodeEngine.getConfigClassLoader()).thenReturn(config.getClassLoader()); - SplitBrainMergePolicyProvider splitBrainMergePolicyProvider = new SplitBrainMergePolicyProvider(nodeEngine); + splitBrainMergePolicyProvider = new SplitBrainMergePolicyProvider(nodeEngine); when(nodeEngine.getSplitBrainMergePolicyProvider()).thenReturn(splitBrainMergePolicyProvider); - mapMergePolicyProvider = new MergePolicyProvider(nodeEngine); - cacheMergePolicyProvider = new CacheMergePolicyProvider(nodeEngine); properties = nodeEngine.getProperties(); } @@ -78,12 +73,12 @@ public void testConstructor() { @Test public void checkMapConfig_BINARY() { - checkMapConfig(getMapConfig(BINARY), nativeMemoryConfig, mapMergePolicyProvider, properties); + checkMapConfig(getMapConfig(BINARY), nativeMemoryConfig, splitBrainMergePolicyProvider, properties); } @Test public void checkMapConfig_OBJECT() { - checkMapConfig(getMapConfig(OBJECT), nativeMemoryConfig, mapMergePolicyProvider, properties); + checkMapConfig(getMapConfig(OBJECT), nativeMemoryConfig, splitBrainMergePolicyProvider, properties); } /** @@ -91,7 +86,7 @@ public void checkMapConfig_OBJECT() { */ @Test(expected = IllegalArgumentException.class) public void checkMapConfig_NATIVE() { - checkMapConfig(getMapConfig(NATIVE), nativeMemoryConfig, mapMergePolicyProvider, properties); + checkMapConfig(getMapConfig(NATIVE), nativeMemoryConfig, splitBrainMergePolicyProvider, properties); } @Test @@ -100,7 +95,7 @@ public void checkMapConfig_withIgnoredConfigMinEvictionCheckMillis() { MapConfig mapConfig = getMapConfig(BINARY) .setMinEvictionCheckMillis(100); - checkMapConfig(mapConfig, nativeMemoryConfig, mapMergePolicyProvider, properties); + checkMapConfig(mapConfig, nativeMemoryConfig, splitBrainMergePolicyProvider, properties); } @Test @@ -109,7 +104,7 @@ public void checkMapConfig_withIgnoredConfigEvictionPercentage() { MapConfig mapConfig = getMapConfig(BINARY) .setEvictionPercentage(50); - checkMapConfig(mapConfig, nativeMemoryConfig, mapMergePolicyProvider, properties); + checkMapConfig(mapConfig, nativeMemoryConfig, splitBrainMergePolicyProvider, properties); } private MapConfig getMapConfig(InMemoryFormat inMemoryFormat) { @@ -125,7 +120,7 @@ public void checkCacheConfig_withEntryCountMaxSizePolicy_OBJECT() { .setInMemoryFormat(OBJECT) .setEvictionConfig(evictionConfig); - checkCacheConfig(cacheSimpleConfig, cacheMergePolicyProvider); + checkCacheConfig(cacheSimpleConfig, splitBrainMergePolicyProvider); } @Test(expected = IllegalArgumentException.class) @@ -136,7 +131,7 @@ public void checkCacheConfig_withEntryCountMaxSizePolicy_NATIVE() { .setInMemoryFormat(NATIVE) .setEvictionConfig(evictionConfig); - checkCacheConfig(cacheSimpleConfig, cacheMergePolicyProvider); + checkCacheConfig(cacheSimpleConfig, splitBrainMergePolicyProvider); } @Test diff --git a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorCacheIntegrationTest.java b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorCacheIntegrationTest.java index 690d82eb4252f..d914308c32c64 100644 --- a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorCacheIntegrationTest.java +++ b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorCacheIntegrationTest.java @@ -16,7 +16,6 @@ package com.hazelcast.internal.config; -import com.hazelcast.cache.merge.PutIfAbsentCacheMergePolicy; import com.hazelcast.config.CacheSimpleConfig; import com.hazelcast.config.Config; import com.hazelcast.config.InvalidConfigurationException; @@ -141,14 +140,4 @@ public void testCache_withCustomMapMergePolicy() { expectedException.expectMessage(containsString(MapMergeTypes.class.getName())); hz.getCacheManager().getCache("customMap"); } - - @Test - public void testCache_withLegacyPutIfAbsentMergePolicy() { - MergePolicyConfig legacyMergePolicyConfig = new MergePolicyConfig() - .setPolicy(PutIfAbsentCacheMergePolicy.class.getName()); - - HazelcastInstance hz = getHazelcastInstance("legacyPutIfAbsent", legacyMergePolicyConfig); - - hz.getCacheManager().getCache("legacyPutIfAbsent"); - } } diff --git a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorCachingProviderIntegrationTest.java b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorCachingProviderIntegrationTest.java index 28727efbe623a..53883d32b5d3f 100644 --- a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorCachingProviderIntegrationTest.java +++ b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorCachingProviderIntegrationTest.java @@ -18,7 +18,6 @@ import com.hazelcast.cache.impl.HazelcastServerCachingProvider; import com.hazelcast.cache.jsr.JsrTestUtil; -import com.hazelcast.cache.merge.PutIfAbsentCacheMergePolicy; import com.hazelcast.config.CacheConfig; import com.hazelcast.config.Config; import com.hazelcast.config.InvalidConfigurationException; @@ -27,6 +26,7 @@ import com.hazelcast.internal.config.mergepolicies.ComplexCustomMergePolicy; import com.hazelcast.spi.merge.MergingCosts; import com.hazelcast.spi.merge.MergingExpirationTime; +import com.hazelcast.spi.merge.PutIfAbsentMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergeTypes.MapMergeTypes; import com.hazelcast.test.HazelcastSerialClassRunner; import com.hazelcast.test.annotation.QuickTest; @@ -46,7 +46,8 @@ */ @RunWith(HazelcastSerialClassRunner.class) @Category(QuickTest.class) -public class MergePolicyValidatorCachingProviderIntegrationTest extends AbstractMergePolicyValidatorIntegrationTest { +public class MergePolicyValidatorCachingProviderIntegrationTest + extends AbstractMergePolicyValidatorIntegrationTest { @BeforeClass public static void jsrSetup() { @@ -151,7 +152,7 @@ public void testCache_withCustomMapMergePolicy() { @Test public void testCache_withLegacyPutIfAbsentMergePolicy() { MergePolicyConfig legacyMergePolicyConfig = new MergePolicyConfig() - .setPolicy(PutIfAbsentCacheMergePolicy.class.getName()); + .setPolicy(PutIfAbsentMergePolicy.class.getName()); getCache("legacyPutIfAbsent", legacyMergePolicyConfig); } diff --git a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorMapIntegrationTest.java b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorMapIntegrationTest.java index 2609160525fa4..5c3ec04f4de25 100644 --- a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorMapIntegrationTest.java +++ b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorMapIntegrationTest.java @@ -22,7 +22,6 @@ import com.hazelcast.core.HazelcastInstance; import com.hazelcast.internal.config.mergepolicies.ComplexCustomMergePolicy; import com.hazelcast.internal.config.mergepolicies.CustomMapMergePolicy; -import com.hazelcast.map.merge.PutIfAbsentMapMergePolicy; import com.hazelcast.spi.merge.MergingCosts; import com.hazelcast.spi.merge.MergingExpirationTime; import com.hazelcast.spi.merge.MergingLastStoredTime; @@ -198,14 +197,4 @@ public void testMap_withCustomMapMergePolicyNoTypeVariable_withStatsEnabled() { hz.getMap("customMapNoTypeVariable"); } - - @Test - public void testMap_withLegacyPutIfAbsentMergePolicy() { - MergePolicyConfig legacyMergePolicyConfig = new MergePolicyConfig() - .setPolicy(PutIfAbsentMapMergePolicy.class.getName()); - - HazelcastInstance hz = getHazelcastInstance("legacyPutIfAbsent", legacyMergePolicyConfig); - - hz.getMap("legacyPutIfAbsent"); - } } diff --git a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorReplicatedMapIntegrationTest.java b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorReplicatedMapIntegrationTest.java index e054adf1a846b..fc9d27c2e9c53 100644 --- a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorReplicatedMapIntegrationTest.java +++ b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorReplicatedMapIntegrationTest.java @@ -20,7 +20,7 @@ import com.hazelcast.config.MergePolicyConfig; import com.hazelcast.config.ReplicatedMapConfig; import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.replicatedmap.merge.PutIfAbsentMapMergePolicy; +import com.hazelcast.spi.merge.PutIfAbsentMergePolicy; import com.hazelcast.test.HazelcastParallelClassRunner; import com.hazelcast.test.annotation.ParallelJVMTest; import com.hazelcast.test.annotation.QuickTest; @@ -77,7 +77,7 @@ public void testReplicatedMap_withInvalidMergePolicy() { @Test public void testReplicatedMap_withLegacyPutIfAbsentMergePolicy() { MergePolicyConfig legacyMergePolicyConfig = new MergePolicyConfig() - .setPolicy(PutIfAbsentMapMergePolicy.class.getName()); + .setPolicy(PutIfAbsentMergePolicy.class.getName()); HazelcastInstance hz = getHazelcastInstance("legacyPutIfAbsent", legacyMergePolicyConfig); diff --git a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorTest.java b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorTest.java index 920e13c0f4f87..cdd6e91fcc8ac 100644 --- a/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorTest.java +++ b/hazelcast/src/test/java/com/hazelcast/internal/config/MergePolicyValidatorTest.java @@ -17,11 +17,8 @@ package com.hazelcast.internal.config; import com.hazelcast.config.Config; -import com.hazelcast.config.InvalidConfigurationException; import com.hazelcast.logging.ILogger; import com.hazelcast.logging.Logger; -import com.hazelcast.map.merge.MergePolicyProvider; -import com.hazelcast.map.merge.PutIfAbsentMapMergePolicy; import com.hazelcast.spi.NodeEngine; import com.hazelcast.spi.merge.PutIfAbsentMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; @@ -38,7 +35,6 @@ import static com.hazelcast.config.InMemoryFormat.NATIVE; import static com.hazelcast.config.InMemoryFormat.OBJECT; import static com.hazelcast.internal.config.MergePolicyValidator.checkMergePolicySupportsInMemoryFormat; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.when; @@ -48,7 +44,7 @@ public class MergePolicyValidatorTest extends HazelcastTestSupport { private static final ILogger LOGGER = Logger.getLogger(MergePolicyValidatorTest.class); - private MergePolicyProvider mapMergePolicyProvider; + private SplitBrainMergePolicyProvider mapMergePolicyProvider; @Before public void setUp() { @@ -56,10 +52,8 @@ public void setUp() { NodeEngine nodeEngine = Mockito.mock(NodeEngine.class); when(nodeEngine.getConfigClassLoader()).thenReturn(config.getClassLoader()); - SplitBrainMergePolicyProvider splitBrainMergePolicyProvider = new SplitBrainMergePolicyProvider(nodeEngine); - when(nodeEngine.getSplitBrainMergePolicyProvider()).thenReturn(splitBrainMergePolicyProvider); - - mapMergePolicyProvider = new MergePolicyProvider(nodeEngine); + mapMergePolicyProvider = new SplitBrainMergePolicyProvider(nodeEngine); + when(nodeEngine.getSplitBrainMergePolicyProvider()).thenReturn(mapMergePolicyProvider); } @Test @@ -73,34 +67,10 @@ public void testCheckMergePolicySupportsInMemoryFormat_withMergePolicy_OBJECT() assertTrue(checkMergePolicySupportsInMemoryFormat("myMap", mergePolicy, OBJECT, false, LOGGER)); } - @Test - public void testCheckMergePolicySupportsInMemoryFormat_withLegacyMergePolicy_OBJECT() { - Object legacyMergePolicy = mapMergePolicyProvider.getMergePolicy(PutIfAbsentMapMergePolicy.class.getName()); - assertTrue(checkMergePolicySupportsInMemoryFormat("myMap", legacyMergePolicy, OBJECT, false, LOGGER)); - } - @Test public void testCheckMergePolicySupportsInMemoryFormat_withMergePolicy_NATIVE() { Object mergePolicy = mapMergePolicyProvider.getMergePolicy(PutIfAbsentMergePolicy.class.getName()); assertTrue(checkMergePolicySupportsInMemoryFormat("myMap", mergePolicy, NATIVE, false, LOGGER)); } - /** - * A legacy merge policy cannot merge NATIVE maps. - */ - @Test - public void testCheckMergePolicySupportsInMemoryFormat_withLegacyMergePolicy_NATIVE() { - Object legacyMergePolicy = mapMergePolicyProvider.getMergePolicy(PutIfAbsentMapMergePolicy.class.getName()); - assertFalse(checkMergePolicySupportsInMemoryFormat("myMap", legacyMergePolicy, NATIVE, false, LOGGER)); - } - - /** - * A legacy merge policy cannot merge NATIVE maps. - */ - @Test(expected = InvalidConfigurationException.class) - public void testCheckMergePolicySupportsInMemoryFormat_withLegacyMergePolicy_NATIVE_failFast() { - Object legacyMergePolicy = mapMergePolicyProvider.getMergePolicy(PutIfAbsentMapMergePolicy.class.getName()); - checkMergePolicySupportsInMemoryFormat("myMap", legacyMergePolicy, NATIVE, true, LOGGER); - } - } diff --git a/hazelcast/src/test/java/com/hazelcast/internal/dynamicconfig/DynamicConfigTest.java b/hazelcast/src/test/java/com/hazelcast/internal/dynamicconfig/DynamicConfigTest.java index ad3206d5bbf44..f3ee9857774ce 100644 --- a/hazelcast/src/test/java/com/hazelcast/internal/dynamicconfig/DynamicConfigTest.java +++ b/hazelcast/src/test/java/com/hazelcast/internal/dynamicconfig/DynamicConfigTest.java @@ -232,8 +232,8 @@ public void testRingbufferConfig() { public void testRingbufferConfig_whenConfiguredWithRingbufferStore_byClassName() { RingbufferConfig config = getRingbufferConfig(); config.getRingbufferStoreConfig() - .setEnabled(true) - .setClassName("com.hazelcast.Foo"); + .setEnabled(true) + .setClassName("com.hazelcast.Foo"); driver.getConfig().addRingBufferConfig(config); @@ -244,8 +244,8 @@ public void testRingbufferConfig_whenConfiguredWithRingbufferStore_byClassName() public void testRingbufferConfig_whenConfiguredWithRingbufferStore_byFactoryClassName() { RingbufferConfig config = getRingbufferConfig(); config.getRingbufferStoreConfig() - .setEnabled(true) - .setFactoryClassName("com.hazelcast.FactoryFoo"); + .setEnabled(true) + .setFactoryClassName("com.hazelcast.FactoryFoo"); driver.getConfig().addRingBufferConfig(config); @@ -256,8 +256,8 @@ public void testRingbufferConfig_whenConfiguredWithRingbufferStore_byFactoryClas public void testRingbufferConfig_whenConfiguredWithRingbufferStore_byStoreImplementation() { RingbufferConfig config = getRingbufferConfig(); config.getRingbufferStoreConfig() - .setEnabled(true) - .setStoreImplementation(new SampleRingbufferStore()); + .setEnabled(true) + .setStoreImplementation(new SampleRingbufferStore()); driver.getConfig().addRingBufferConfig(config); @@ -268,8 +268,8 @@ public void testRingbufferConfig_whenConfiguredWithRingbufferStore_byStoreImplem public void testRingbufferConfig_whenConfiguredWithRingbufferStore_byFactoryImplementation() { RingbufferConfig config = getRingbufferConfig(); config.getRingbufferStoreConfig() - .setEnabled(true) - .setFactoryImplementation(new SampleRingbufferStoreFactory()); + .setEnabled(true) + .setFactoryImplementation(new SampleRingbufferStoreFactory()); driver.getConfig().addRingBufferConfig(config); @@ -326,9 +326,9 @@ public void testSameReplicatedMapConfig_canBeAddedTwice() { public void testReplicatedMapConfig_withListenerByClassName() { ReplicatedMapConfig config = new ReplicatedMapConfig(name) .setStatisticsEnabled(true) - .setMergePolicy("com.hazelcast.SomeMergePolicy") .setInMemoryFormat(InMemoryFormat.NATIVE) .addEntryListenerConfig(new EntryListenerConfig(randomString(), true, false)); + config.getMergePolicyConfig().setPolicy("com.hazelcast.SomeMergePolicy"); config.setAsyncFillup(true); driver.getConfig().addReplicatedMapConfig(config); @@ -340,9 +340,9 @@ public void testReplicatedMapConfig_withListenerByClassName() { public void testReplicatedMapConfig_withListenerByImplementation() { ReplicatedMapConfig config = new ReplicatedMapConfig(name) .setStatisticsEnabled(true) - .setMergePolicy("com.hazelcast.SomeMergePolicy") .setInMemoryFormat(InMemoryFormat.NATIVE) .addEntryListenerConfig(new EntryListenerConfig(new SampleEntryListener(), false, true)); + config.getMergePolicyConfig().setPolicy("com.hazelcast.SomeMergePolicy"); config.setAsyncFillup(true); driver.getConfig().addReplicatedMapConfig(config); diff --git a/hazelcast/src/test/java/com/hazelcast/map/MapMergePolicyQuickTest.java b/hazelcast/src/test/java/com/hazelcast/map/MapMergePolicyQuickTest.java index 19b390a3e2cc2..95f20878606f5 100644 --- a/hazelcast/src/test/java/com/hazelcast/map/MapMergePolicyQuickTest.java +++ b/hazelcast/src/test/java/com/hazelcast/map/MapMergePolicyQuickTest.java @@ -21,12 +21,14 @@ import com.hazelcast.map.impl.MapServiceContext; import com.hazelcast.map.impl.SimpleEntryView; import com.hazelcast.map.impl.recordstore.RecordStore; -import com.hazelcast.map.merge.LatestUpdateMapMergePolicy; -import com.hazelcast.map.merge.MapMergePolicy; -import com.hazelcast.map.merge.PassThroughMergePolicy; -import com.hazelcast.map.merge.PutIfAbsentMapMergePolicy; import com.hazelcast.nio.serialization.Data; +import com.hazelcast.spi.NodeEngine; import com.hazelcast.spi.impl.NodeEngineImpl; +import com.hazelcast.spi.merge.LatestUpdateMergePolicy; +import com.hazelcast.spi.merge.PassThroughMergePolicy; +import com.hazelcast.spi.merge.PutIfAbsentMergePolicy; +import com.hazelcast.spi.merge.SplitBrainMergePolicy; +import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; import com.hazelcast.test.HazelcastParallelClassRunner; import com.hazelcast.test.HazelcastTestSupport; import com.hazelcast.test.annotation.ParallelJVMTest; @@ -36,6 +38,7 @@ import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; +import static com.hazelcast.spi.impl.merge.MergingValueFactory.createMergingEntry; import static org.junit.Assert.assertEquals; @RunWith(HazelcastParallelClassRunner.class) @@ -50,23 +53,27 @@ public void testLatestUpdateMapMergePolicy() { MapServiceContext mapServiceContext = getMapServiceContext(instance); Data dataKey = mapServiceContext.toData("key"); + Data dataValue = mapServiceContext.toData("value1"); + Data dataValue2 = mapServiceContext.toData("value2"); RecordStore recordStore = mapServiceContext.getRecordStore(getPartitionId(instance, "key"), name); - MapMergePolicy mergePolicy = (MapMergePolicy) mapServiceContext.getMergePolicyProvider() - .getMergePolicy(LatestUpdateMapMergePolicy.class.getName()); + NodeEngine nodeEngine = mapServiceContext.getNodeEngine(); + SplitBrainMergePolicyProvider mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); + SplitBrainMergePolicy mergePolicy = mergePolicyProvider.getMergePolicy(LatestUpdateMergePolicy.class.getName()); long now = Clock.currentTimeMillis(); - SimpleEntryView initialEntry = new SimpleEntryView("key", "value1"); + + SimpleEntryView initialEntry = new SimpleEntryView<>(dataKey, dataValue); initialEntry.setCreationTime(now); initialEntry.setLastUpdateTime(now); // need some latency to be sure that target members time is greater than now sleepMillis(100); - recordStore.merge(dataKey, initialEntry, mergePolicy); + recordStore.merge(createMergingEntry(nodeEngine.getSerializationService(), initialEntry), mergePolicy); - SimpleEntryView mergingEntry = new SimpleEntryView("key", "value2"); + SimpleEntryView mergingEntry = new SimpleEntryView<>(dataKey, dataValue2); now = Clock.currentTimeMillis(); mergingEntry.setCreationTime(now); mergingEntry.setLastUpdateTime(now); - recordStore.merge(dataKey, mergingEntry, mergePolicy); + recordStore.merge(createMergingEntry(nodeEngine.getSerializationService(), mergingEntry), mergePolicy); assertEquals("value2", map.get("key")); } @@ -79,16 +86,19 @@ public void testPutIfAbsentMapMergePolicy() { MapServiceContext mapServiceContext = getMapServiceContext(instance); Data dataKey = mapServiceContext.toData("key"); + Data dataValue = mapServiceContext.toData("value1"); + Data dataValue2 = mapServiceContext.toData("value2"); RecordStore recordStore = mapServiceContext.getRecordStore(getPartitionId(instance, "key"), name); - MapMergePolicy mergePolicy = (MapMergePolicy) mapServiceContext.getMergePolicyProvider() - .getMergePolicy(PutIfAbsentMapMergePolicy.class.getName()); + NodeEngine nodeEngine = mapServiceContext.getNodeEngine(); + SplitBrainMergePolicyProvider mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); + SplitBrainMergePolicy mergePolicy = mergePolicyProvider.getMergePolicy(PutIfAbsentMergePolicy.class.getName()); - SimpleEntryView initialEntry = new SimpleEntryView("key", "value1"); - recordStore.merge(dataKey, initialEntry, mergePolicy); + SimpleEntryView initialEntry = new SimpleEntryView<>(dataKey, dataValue); + recordStore.merge(createMergingEntry(nodeEngine.getSerializationService(), initialEntry), mergePolicy); - SimpleEntryView mergingEntry = new SimpleEntryView("key", "value2"); - recordStore.merge(dataKey, mergingEntry, mergePolicy); + SimpleEntryView mergingEntry = new SimpleEntryView<>(dataKey, dataValue2); + recordStore.merge(createMergingEntry(nodeEngine.getSerializationService(), mergingEntry), mergePolicy); assertEquals("value1", map.get("key")); } @@ -101,15 +111,19 @@ public void testPassThroughMapMergePolicy() { MapServiceContext mapServiceContext = getMapServiceContext(instance); Data dataKey = mapServiceContext.toData("key"); + Data dataValue = mapServiceContext.toData("value1"); + Data dataValue2 = mapServiceContext.toData("value2"); RecordStore recordStore = mapServiceContext.getRecordStore(getPartitionId(instance, "key"), name); - MapMergePolicy mergePolicy = (MapMergePolicy) mapServiceContext.getMergePolicyProvider() - .getMergePolicy(PassThroughMergePolicy.class.getName()); - SimpleEntryView initialEntry = new SimpleEntryView("key", "value1"); - recordStore.merge(dataKey, initialEntry, mergePolicy); + NodeEngine nodeEngine = mapServiceContext.getNodeEngine(); + SplitBrainMergePolicyProvider mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); + SplitBrainMergePolicy mergePolicy = mergePolicyProvider.getMergePolicy(PassThroughMergePolicy.class.getName()); + + SimpleEntryView initialEntry = new SimpleEntryView<>(dataKey, dataValue); + recordStore.merge(createMergingEntry(nodeEngine.getSerializationService(), initialEntry), mergePolicy); - SimpleEntryView mergingEntry = new SimpleEntryView("key", "value2"); - recordStore.merge(dataKey, mergingEntry, mergePolicy); + SimpleEntryView mergingEntry = new SimpleEntryView<>(dataKey, dataValue2); + recordStore.merge(createMergingEntry(nodeEngine.getSerializationService(), mergingEntry), mergePolicy); assertEquals("value2", map.get("key")); } diff --git a/hazelcast/src/test/java/com/hazelcast/map/MapMergePolicySerializationTest.java b/hazelcast/src/test/java/com/hazelcast/map/MapMergePolicySerializationTest.java index 13989cfc56d39..1960edf2cd6d3 100644 --- a/hazelcast/src/test/java/com/hazelcast/map/MapMergePolicySerializationTest.java +++ b/hazelcast/src/test/java/com/hazelcast/map/MapMergePolicySerializationTest.java @@ -22,13 +22,14 @@ import com.hazelcast.map.impl.MapServiceContext; import com.hazelcast.map.impl.SimpleEntryView; import com.hazelcast.map.impl.recordstore.RecordStore; -import com.hazelcast.map.merge.MapMergePolicy; -import com.hazelcast.map.merge.PutIfAbsentMapMergePolicy; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.Data; import com.hazelcast.nio.serialization.DataSerializable; import com.hazelcast.spi.impl.NodeEngineImpl; +import com.hazelcast.spi.merge.PutIfAbsentMergePolicy; +import com.hazelcast.spi.merge.SplitBrainMergePolicy; +import com.hazelcast.spi.merge.SplitBrainMergePolicyProvider; import com.hazelcast.test.HazelcastParallelClassRunner; import com.hazelcast.test.HazelcastTestSupport; import com.hazelcast.test.annotation.ParallelJVMTest; @@ -39,6 +40,7 @@ import java.io.IOException; +import static com.hazelcast.spi.impl.merge.MergingValueFactory.createMergingEntry; import static org.junit.Assert.assertEquals; @RunWith(HazelcastParallelClassRunner.class) @@ -59,13 +61,16 @@ public void testIssue2665() { MapService mapService = nodeEngine.getService(serviceName); MapServiceContext mapServiceContext = mapService.getMapServiceContext(); int partitionId = nodeEngine.getPartitionService().getPartitionId("key"); + + MyObject myObject = new MyObject(); Data dataKey = mapServiceContext.toData("key"); + Data dataValue = mapServiceContext.toData(myObject); RecordStore recordStore = mapServiceContext.getRecordStore(partitionId, name); - MapMergePolicy mergePolicy = (MapMergePolicy) mapServiceContext.getMergePolicyProvider() - .getMergePolicy(PutIfAbsentMapMergePolicy.class.getName()); - EntryView mergingEntryView = new SimpleEntryView("key", new MyObject()); - recordStore.merge(dataKey, mergingEntryView, mergePolicy); + SplitBrainMergePolicyProvider mergePolicyProvider = nodeEngine.getSplitBrainMergePolicyProvider(); + SplitBrainMergePolicy mergePolicy = mergePolicyProvider.getMergePolicy(PutIfAbsentMergePolicy.class.getName()); + EntryView mergingEntryView = new SimpleEntryView<>(dataKey, dataValue); + recordStore.merge(createMergingEntry(nodeEngine.getSerializationService(), mergingEntryView), mergePolicy); int deSerializedCount = MyObject.deserializedCount; assertEquals(0, deSerializedCount); diff --git a/hazelcast/src/test/java/com/hazelcast/map/MergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/map/MergePolicyTest.java index 922df3aa4dcb9..b3b96816bcc19 100644 --- a/hazelcast/src/test/java/com/hazelcast/map/MergePolicyTest.java +++ b/hazelcast/src/test/java/com/hazelcast/map/MergePolicyTest.java @@ -16,19 +16,19 @@ package com.hazelcast.map; +import com.hazelcast.cluster.MemberAttributeEvent; +import com.hazelcast.cluster.MembershipEvent; +import com.hazelcast.cluster.MembershipListener; import com.hazelcast.config.Config; import com.hazelcast.core.Hazelcast; import com.hazelcast.core.HazelcastInstance; import com.hazelcast.core.LifecycleEvent; import com.hazelcast.core.LifecycleListener; -import com.hazelcast.cluster.MemberAttributeEvent; -import com.hazelcast.cluster.MembershipEvent; -import com.hazelcast.cluster.MembershipListener; import com.hazelcast.instance.impl.HazelcastInstanceFactory; -import com.hazelcast.map.merge.HigherHitsMapMergePolicy; -import com.hazelcast.map.merge.LatestUpdateMapMergePolicy; -import com.hazelcast.map.merge.PassThroughMergePolicy; -import com.hazelcast.map.merge.PutIfAbsentMapMergePolicy; +import com.hazelcast.spi.merge.HigherHitsMergePolicy; +import com.hazelcast.spi.merge.LatestUpdateMergePolicy; +import com.hazelcast.spi.merge.PassThroughMergePolicy; +import com.hazelcast.spi.merge.PutIfAbsentMergePolicy; import com.hazelcast.spi.properties.GroupProperty; import com.hazelcast.test.HazelcastSerialClassRunner; import com.hazelcast.test.HazelcastTestSupport; @@ -60,7 +60,7 @@ public void killAllHazelcastInstances() { @Test public void testLatestUpdateMapMergePolicy() { String mapName = randomMapName(); - Config config = newConfig(LatestUpdateMapMergePolicy.class.getName(), mapName); + Config config = newConfig(LatestUpdateMergePolicy.class.getName(), mapName); HazelcastInstance h1 = Hazelcast.newHazelcastInstance(config); HazelcastInstance h2 = Hazelcast.newHazelcastInstance(config); @@ -104,7 +104,7 @@ public void testLatestUpdateMapMergePolicy() { @Test public void testHigherHitsMapMergePolicy() { String mapName = randomMapName(); - Config config = newConfig(HigherHitsMapMergePolicy.class.getName(), mapName); + Config config = newConfig(HigherHitsMergePolicy.class.getName(), mapName); HazelcastInstance h1 = Hazelcast.newHazelcastInstance(config); HazelcastInstance h2 = Hazelcast.newHazelcastInstance(config); @@ -151,7 +151,7 @@ public void testHigherHitsMapMergePolicy() { @Test public void testPutIfAbsentMapMergePolicy() { String mapName = randomMapName(); - Config config = newConfig(PutIfAbsentMapMergePolicy.class.getName(), mapName); + Config config = newConfig(PutIfAbsentMergePolicy.class.getName(), mapName); HazelcastInstance h1 = Hazelcast.newHazelcastInstance(config); HazelcastInstance h2 = Hazelcast.newHazelcastInstance(config); @@ -276,7 +276,7 @@ private Config newConfig(String mergePolicy, String mapName) { .setName(generateRandomString(10)); config.getMapConfig(mapName) - .setMergePolicy(mergePolicy); + .getMergePolicyConfig().setPolicy(mergePolicy); return config; } diff --git a/hazelcast/src/test/java/com/hazelcast/map/TestCustomMapMergePolicy.java b/hazelcast/src/test/java/com/hazelcast/map/TestCustomMapMergePolicy.java index d194235c62c84..858acbb54e683 100644 --- a/hazelcast/src/test/java/com/hazelcast/map/TestCustomMapMergePolicy.java +++ b/hazelcast/src/test/java/com/hazelcast/map/TestCustomMapMergePolicy.java @@ -16,19 +16,19 @@ package com.hazelcast.map; -import com.hazelcast.core.EntryView; -import com.hazelcast.map.merge.MapMergePolicy; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; +import com.hazelcast.spi.merge.MergingValue; +import com.hazelcast.spi.merge.SplitBrainMergePolicy; import java.io.IOException; -class TestCustomMapMergePolicy implements MapMergePolicy { +class TestCustomMapMergePolicy implements SplitBrainMergePolicy> { @Override - public Object merge(String mapName, EntryView mergingEntry, EntryView existingEntry) { - if (mergingEntry.getValue() instanceof Integer) { - return mergingEntry.getValue(); + public Integer merge(MergingValue mergingEntry, MergingValue existingEntry) { + if (mergingEntry.getDeserializedValue() instanceof Integer) { + return mergingEntry.getDeserializedValue(); } return null; } diff --git a/hazelcast/src/test/java/com/hazelcast/map/impl/querycache/QueryCacheIMapEventHandlingTest.java b/hazelcast/src/test/java/com/hazelcast/map/impl/querycache/QueryCacheIMapEventHandlingTest.java index 7b24ac7fb8a76..212b5b495b5f9 100644 --- a/hazelcast/src/test/java/com/hazelcast/map/impl/querycache/QueryCacheIMapEventHandlingTest.java +++ b/hazelcast/src/test/java/com/hazelcast/map/impl/querycache/QueryCacheIMapEventHandlingTest.java @@ -17,21 +17,22 @@ package com.hazelcast.map.impl.querycache; import com.hazelcast.core.EntryEvent; -import com.hazelcast.core.EntryView; import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.map.IMap; import com.hazelcast.instance.impl.Node; +import com.hazelcast.map.IMap; import com.hazelcast.map.QueryCache; -import com.hazelcast.map.impl.operation.LegacyMergeOperation; +import com.hazelcast.map.impl.operation.MergeOperation; import com.hazelcast.map.impl.record.Record; import com.hazelcast.map.listener.EntryAddedListener; import com.hazelcast.map.listener.EntryRemovedListener; -import com.hazelcast.map.merge.PassThroughMergePolicy; import com.hazelcast.nio.serialization.Data; import com.hazelcast.query.Predicate; import com.hazelcast.query.Predicates; import com.hazelcast.spi.impl.NodeEngineImpl; +import com.hazelcast.spi.impl.operationservice.Operation; import com.hazelcast.spi.impl.operationservice.impl.OperationServiceImpl; +import com.hazelcast.spi.merge.PassThroughMergePolicy; +import com.hazelcast.spi.merge.SplitBrainMergeTypes; import com.hazelcast.spi.serialization.SerializationService; import com.hazelcast.test.AssertTask; import com.hazelcast.test.HazelcastParallelClassRunner; @@ -47,9 +48,10 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; -import static com.hazelcast.map.impl.EntryViews.createSimpleEntryView; import static com.hazelcast.map.impl.MapService.SERVICE_NAME; import static com.hazelcast.map.impl.querycache.AbstractQueryCacheTestSupport.getMap; +import static com.hazelcast.spi.impl.merge.MergingValueFactory.createMergingEntry; +import static java.util.Collections.singletonList; import static java.util.concurrent.TimeUnit.SECONDS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -99,14 +101,16 @@ public void run() throws Exception { private void executeMergeOperation(HazelcastInstance member, String mapName, int key, int mergedValue) throws Exception { Node node = getNode(member); NodeEngineImpl nodeEngine = node.nodeEngine; - OperationServiceImpl operationService = (OperationServiceImpl) nodeEngine.getOperationService(); + OperationServiceImpl operationService = nodeEngine.getOperationService(); SerializationService serializationService = getSerializationService(member); Data keyData = serializationService.toData(key); Data valueData = serializationService.toData(mergedValue); - EntryView entryView = createSimpleEntryView(keyData, valueData, Mockito.mock(Record.class)); + SplitBrainMergeTypes.MapMergeTypes mergingEntry + = createMergingEntry(serializationService, keyData, valueData, Mockito.mock(Record.class)); - LegacyMergeOperation mergeOperation = new LegacyMergeOperation(mapName, entryView, new PassThroughMergePolicy(), false); + Operation mergeOperation = new MergeOperation(mapName, singletonList(mergingEntry), + new PassThroughMergePolicy<>(), false); int partitionId = nodeEngine.getPartitionService().getPartitionId(key); Future future = operationService.invokeOnPartition(SERVICE_NAME, mergeOperation, partitionId); future.get(); diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/AbstractMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/AbstractMapMergePolicyTest.java deleted file mode 100644 index 60ea5bb6a8494..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/AbstractMapMergePolicyTest.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.core.EntryView; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public abstract class AbstractMapMergePolicyTest { - - private static final String EXISTING = "EXISTING"; - private static final String MERGING = "MERGING"; - - protected MapMergePolicy policy; - - @Test - public void merge_mergingWins() { - EntryView existing = entryWithGivenPropertyAndValue(1, EXISTING); - EntryView merging = entryWithGivenPropertyAndValue(333, MERGING); - - assertEquals(MERGING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_existingWins() { - EntryView existing = entryWithGivenPropertyAndValue(333, EXISTING); - EntryView merging = entryWithGivenPropertyAndValue(1, MERGING); - - assertEquals(EXISTING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_draw_mergingWins() { - EntryView existing = entryWithGivenPropertyAndValue(1, EXISTING); - EntryView merging = entryWithGivenPropertyAndValue(1, MERGING); - - assertEquals(MERGING, policy.merge("map", merging, existing)); - } - - private EntryView entryWithGivenPropertyAndValue(long testedProperty, String value) { - EntryView entryView = mock(EntryView.class); - try { - if (policy instanceof HigherHitsMapMergePolicy) { - when(entryView.getHits()).thenReturn(testedProperty); - } else if (policy instanceof LatestUpdateMapMergePolicy) { - when(entryView.getLastUpdateTime()).thenReturn(testedProperty); - } - when(entryView.getValue()).thenReturn(value); - return entryView; - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/HigherHitsMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/HigherHitsMapMergePolicyTest.java deleted file mode 100644 index f495eb70c8c1b..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/HigherHitsMapMergePolicyTest.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class HigherHitsMapMergePolicyTest extends AbstractMapMergePolicyTest { - - @Before - public void given() { - policy = new HigherHitsMapMergePolicy(); - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/IgnoreMergingEntriesMapSplitBrainTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/IgnoreMergingEntriesMapSplitBrainTest.java deleted file mode 100644 index 82eb284d4244e..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/IgnoreMergingEntriesMapSplitBrainTest.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.config.Config; -import com.hazelcast.config.MapConfig; -import com.hazelcast.config.MergePolicyConfig; -import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.map.IMap; -import com.hazelcast.core.LifecycleEvent; -import com.hazelcast.core.LifecycleListener; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.SplitBrainTestSupport; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import java.util.concurrent.CountDownLatch; - -import static org.junit.Assert.assertTrue; - -/** - * Before merging, put some entries to the merging sub-cluster and expect not to see them after merge. - */ -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class IgnoreMergingEntriesMapSplitBrainTest extends SplitBrainTestSupport { - - private final String testMapName = randomMapName(); - private final CountDownLatch clusterMergedLatch = new CountDownLatch(1); - - @Override - protected Config config() { - MergePolicyConfig mergePolicyConfig = new MergePolicyConfig() - .setPolicy(IgnoreMergingEntryMapMergePolicy.class.getName()); - - MapConfig mapConfig = new MapConfig(testMapName) - .setMergePolicyConfig(mergePolicyConfig); - - return super.config() - .addMapConfig(mapConfig); - } - - @Override - protected int[] brains() { - // first half merges into second half - return new int[]{1, 2}; - } - - @Override - protected void onBeforeSplitBrainCreated(HazelcastInstance[] instances) { - instances[0].getLifecycleService().addLifecycleListener(new MergedLifecycleListener()); - } - - @Override - protected void onAfterSplitBrainCreated(HazelcastInstance[] firstBrain, HazelcastInstance[] secondBrain) { - IMap mapOnFirstBrain = firstBrain[0].getMap(testMapName); - for (int i = 0; i < 100; i++) { - mapOnFirstBrain.put(i, i); - } - } - - @Override - protected void onAfterSplitBrainHealed(HazelcastInstance[] instances) { - assertOpenEventually(clusterMergedLatch, 30); - - IMap map = instances[0].getMap(testMapName); - assertTrue(map.isEmpty()); - } - - class MergedLifecycleListener implements LifecycleListener { - - @Override - public void stateChanged(LifecycleEvent event) { - if (event.getState() == LifecycleEvent.LifecycleState.MERGED) { - clusterMergedLatch.countDown(); - } - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/LatestUpdateMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/LatestUpdateMapMergePolicyTest.java deleted file mode 100644 index 3e546a53a383e..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/LatestUpdateMapMergePolicyTest.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class LatestUpdateMapMergePolicyTest extends AbstractMapMergePolicyTest { - - @Before - public void given() { - policy = new LatestUpdateMapMergePolicy(); - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/LegacyCustomMapSplitBrainTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/LegacyCustomMapSplitBrainTest.java deleted file mode 100644 index c27326ec1b816..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/LegacyCustomMapSplitBrainTest.java +++ /dev/null @@ -1,176 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.config.Config; -import com.hazelcast.core.EntryView; -import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.map.IMap; -import com.hazelcast.core.LifecycleEvent; -import com.hazelcast.core.LifecycleListener; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.SplitBrainTestSupport; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import java.io.IOException; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.junit.Assert.assertEquals; - -/** - * Given: - * Cluster with 3 members, maps configured with a custom merge policy that subtracts merging from existing value (if exists) - * or the merging value itself. - *

- * When: - * Cluster splits in two sub-clusters with {1, 2} members respectively, on each brain put values: - *

    - *
  • on first brain, keys 0..1999 -> value 1
  • - *
  • on second brain, keys 1000..2999 -> value 3
  • - *
- *

- * Then: - * Custom merge policy's merge() method is invoked for all entries of the map, assert final map values as follows: - *

    - *
  • keys 0..999 -> value 1 (merged, no existing value)
  • - *
  • keys 1000..1999 -> value 2 (merged, result of (3-1))
  • - *
  • keys 2000..2999 -> value 3 (not merged)
  • - *
- */ -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class LegacyCustomMapSplitBrainTest extends SplitBrainTestSupport { - - private static final String TEST_MAPS_PREFIX = "MapSplitBrainTest"; - private static final CopyOnWriteArrayList MERGE_POLICY_INSTANCES - = new CopyOnWriteArrayList(); - - private final CountDownLatch clusterMergedLatch = new CountDownLatch(1); - private final AtomicInteger countOfMerges = new AtomicInteger(); - - private String testMapName; - - @Override - protected int[] brains() { - // first half merges into second half - return new int[]{1, 2}; - } - - @Override - protected Config config() { - Config config = super.config(); - config.getMapConfig(TEST_MAPS_PREFIX + "*") - .setMergePolicy(SubtractingMergePolicy.class.getName()); - return config; - } - - @Override - protected void onBeforeSplitBrainCreated(HazelcastInstance[] instances) { - testMapName = TEST_MAPS_PREFIX + randomMapName(); - - instances[0].getLifecycleService().addLifecycleListener(new MergedLifecycleListener()); - } - - @Override - protected void onAfterSplitBrainCreated(HazelcastInstance[] firstBrain, HazelcastInstance[] secondBrain) { - // put value 1 for keys 0..1999 on first brain - IMap mapOnFirstBrain = firstBrain[0].getMap(testMapName); - for (int i = 0; i < 2000; i++) { - mapOnFirstBrain.put(i, 1); - } - - // put value 3 for keys 1000..2999 on second brain - IMap mapOnSecondBrain = secondBrain[0].getMap(testMapName); - for (int i = 1000; i < 3000; i++) { - mapOnSecondBrain.put(i, 3); - } - } - - @Override - protected void onAfterSplitBrainHealed(HazelcastInstance[] instances) { - assertOpenEventually(clusterMergedLatch, 30); - // map on smaller, merging cluster has 2000 entries (0..1999), so 2000 merges should have happened - assertEquals(2000, countOfMerges.get()); - IMap map = instances[0].getMap(testMapName); - // final map should have: - // keys 0..999: value 1 - // keys 1000..1999: value 2 - // keys 2000..2999: value 3 - for (int i = 0; i < 3000; i++) { - try { - assertEquals(i / 1000 + 1, (long) map.get(i)); - } catch (Exception e) { - System.out.println(">>>> " + i); - e.printStackTrace(); - } - } - } - - /** - * Subtracts the integer value of the merging entry from the existing entry (if one exists). - */ - public static class SubtractingMergePolicy implements MapMergePolicy { - - final AtomicInteger counter; - - public SubtractingMergePolicy() { - this.counter = new AtomicInteger(); - MERGE_POLICY_INSTANCES.add(this); - } - - @Override - public void writeData(ObjectDataOutput out) - throws IOException { - } - - @Override - public void readData(ObjectDataInput in) - throws IOException { - } - - @Override - public Object merge(String mapName, EntryView mergingEntry, EntryView existingEntry) { - counter.incrementAndGet(); - Integer existingValue = (Integer) existingEntry.getValue(); - Integer mergingValue = (Integer) mergingEntry.getValue(); - if (existingValue != null) { - return existingValue - mergingValue; - } else { - return mergingEntry.getValue(); - } - } - } - - class MergedLifecycleListener implements LifecycleListener { - @Override - public void stateChanged(LifecycleEvent event) { - if (event.getState() == LifecycleEvent.LifecycleState.MERGED) { - for (SubtractingMergePolicy mergePolicy : MERGE_POLICY_INSTANCES) { - countOfMerges.addAndGet(mergePolicy.counter.get()); - } - clusterMergedLatch.countDown(); - } - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/LegacyMapSplitBrainTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/LegacyMapSplitBrainTest.java deleted file mode 100644 index 4df468f2cf19e..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/LegacyMapSplitBrainTest.java +++ /dev/null @@ -1,368 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.config.Config; -import com.hazelcast.config.InMemoryFormat; -import com.hazelcast.config.MergePolicyConfig; -import com.hazelcast.core.EntryView; -import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.map.IMap; -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.nio.serialization.DataSerializable; -import com.hazelcast.test.HazelcastParallelParametersRunnerFactory; -import com.hazelcast.test.SplitBrainTestSupport; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import com.hazelcast.test.backup.BackupAccessor; -import com.hazelcast.test.backup.TestBackupUtils; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; -import org.junit.runners.Parameterized.UseParametersRunnerFactory; - -import java.util.Collection; - -import static com.hazelcast.config.InMemoryFormat.BINARY; -import static com.hazelcast.config.InMemoryFormat.OBJECT; -import static com.hazelcast.test.backup.TestBackupUtils.assertBackupEntryEqualsEventually; -import static com.hazelcast.test.backup.TestBackupUtils.assertBackupEntryNullEventually; -import static com.hazelcast.test.backup.TestBackupUtils.assertBackupSizeEventually; -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * Tests different split-brain scenarios for {@link IMap}. - *

- * Most merge policies are tested with {@link InMemoryFormat#BINARY} only, since they don't check the value. - *

- * The {@link CustomLegacyMergePolicy} is tested with both in-memory formats, since it's using the value to merge. - *

- * The {@link IgnoreMergingEntryMapMergePolicy}, {@link PassThroughMergePolicy} and {@link PutIfAbsentMapMergePolicy} are also - * tested with a data structure, which is only created in the smaller cluster. - */ -@RunWith(Parameterized.class) -@UseParametersRunnerFactory(HazelcastParallelParametersRunnerFactory.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -@SuppressWarnings("WeakerAccess") -public class LegacyMapSplitBrainTest extends SplitBrainTestSupport { - - @Parameters(name = "format:{0}, mergePolicy:{1}") - public static Collection parameters() { - return asList(new Object[][]{ - {BINARY, IgnoreMergingEntryMapMergePolicy.class}, - {BINARY, HigherHitsMapMergePolicy.class}, - {BINARY, LatestUpdateMapMergePolicy.class}, - {BINARY, PassThroughMergePolicy.class}, - {BINARY, PutIfAbsentMapMergePolicy.class}, - - {BINARY, CustomLegacyMergePolicy.class}, - {OBJECT, CustomLegacyMergePolicy.class}, - }); - } - - @Parameter - public InMemoryFormat inMemoryFormat; - - @Parameter(value = 1) - public Class mergePolicyClass; - - protected String mapNameA = randomMapName("mapA-"); - protected String mapNameB = randomMapName("mapB-"); - private IMap mapA1; - private IMap mapA2; - private IMap mapB1; - private IMap mapB2; - private BackupAccessor backupMapA; - private BackupAccessor backupMapB; - private MergeLifecycleListener mergeLifecycleListener; - - @Override - protected Config config() { - MergePolicyConfig mergePolicyConfig = new MergePolicyConfig() - .setPolicy(mergePolicyClass.getName()) - .setBatchSize(10); - - Config config = super.config(); - config.getMapConfig(mapNameA) - .setInMemoryFormat(inMemoryFormat) - .setMergePolicyConfig(mergePolicyConfig) - .setBackupCount(1) - .setAsyncBackupCount(0); - config.getMapConfig(mapNameB) - .setInMemoryFormat(inMemoryFormat) - .setMergePolicyConfig(mergePolicyConfig) - .setBackupCount(1) - .setAsyncBackupCount(0); - return config; - } - - @Override - protected void onBeforeSplitBrainCreated(HazelcastInstance[] instances) { - waitAllForSafeState(instances); - - BackupAccessor accessor = TestBackupUtils.newMapAccessor(instances, mapNameA); - assertEquals("backupMap should contain 0 entries", 0, accessor.size()); - } - - @Override - protected void onAfterSplitBrainCreated(HazelcastInstance[] firstBrain, HazelcastInstance[] secondBrain) { - mergeLifecycleListener = new MergeLifecycleListener(secondBrain.length); - for (HazelcastInstance instance : secondBrain) { - instance.getLifecycleService().addLifecycleListener(mergeLifecycleListener); - } - - mapA1 = firstBrain[0].getMap(mapNameA); - mapA2 = secondBrain[0].getMap(mapNameA); - mapB2 = secondBrain[0].getMap(mapNameB); - - if (mergePolicyClass == IgnoreMergingEntryMapMergePolicy.class) { - afterSplitDiscardMergePolicy(); - } else if (mergePolicyClass == HigherHitsMapMergePolicy.class) { - afterSplitHigherHitsMergePolicy(); - } else if (mergePolicyClass == LatestUpdateMapMergePolicy.class) { - afterSplitLatestUpdateMergePolicy(); - } else if (mergePolicyClass == PassThroughMergePolicy.class) { - afterSplitPassThroughMergePolicy(); - } else if (mergePolicyClass == PutIfAbsentMapMergePolicy.class) { - afterSplitPutIfAbsentMergePolicy(); - } else if (mergePolicyClass == CustomLegacyMergePolicy.class) { - afterSplitCustomMergePolicy(); - } else { - fail(); - } - } - - @Override - protected void onAfterSplitBrainHealed(HazelcastInstance[] instances) { - // wait until merge completes - mergeLifecycleListener.await(); - - mapB1 = instances[0].getMap(mapNameB); - - backupMapA = TestBackupUtils.newMapAccessor(instances, mapNameA); - backupMapB = TestBackupUtils.newMapAccessor(instances, mapNameB); - - if (mergePolicyClass == IgnoreMergingEntryMapMergePolicy.class) { - afterMergeDiscardMergePolicy(); - } else if (mergePolicyClass == HigherHitsMapMergePolicy.class) { - afterMergeHigherHitsMergePolicy(); - } else if (mergePolicyClass == LatestUpdateMapMergePolicy.class) { - afterMergeLatestUpdateMergePolicy(); - } else if (mergePolicyClass == PassThroughMergePolicy.class) { - afterMergePassThroughMergePolicy(); - } else if (mergePolicyClass == PutIfAbsentMapMergePolicy.class) { - afterMergePutIfAbsentMergePolicy(); - } else if (mergePolicyClass == CustomLegacyMergePolicy.class) { - afterMergeCustomMergePolicy(); - } else { - fail(); - } - } - - private void afterSplitDiscardMergePolicy() { - mapA1.put("key1", "value1"); - - mapA2.put("key1", "DiscardedValue1"); - mapA2.put("key2", "DiscardedValue2"); - - mapB2.put("key", "DiscardedValue"); - } - - private void afterMergeDiscardMergePolicy() { - assertEquals("value1", mapA1.get("key1")); - assertEquals("value1", mapA2.get("key1")); - assertBackupEntryEqualsEventually("key1", "value1", backupMapA); - - assertNull(mapA1.get("key2")); - assertNull(mapA2.get("key2")); - assertBackupEntryNullEventually("key2", backupMapA); - - assertEquals(1, mapA1.size()); - assertEquals(1, mapA2.size()); - assertBackupSizeEventually(1, backupMapA); - - assertNull(mapB1.get("key")); - assertNull(mapB2.get("key")); - assertBackupEntryNullEventually("key", backupMapB); - - assertTrue(mapB1.isEmpty()); - assertTrue(mapB2.isEmpty()); - assertBackupSizeEventually(0, backupMapB); - } - - private void afterSplitHigherHitsMergePolicy() { - mapA1.put("key1", "higherHitsValue1"); - mapA1.put("key2", "value2"); - - // increase hits number - assertEquals("higherHitsValue1", mapA1.get("key1")); - assertEquals("higherHitsValue1", mapA1.get("key1")); - - mapA2.put("key1", "value1"); - mapA2.put("key2", "higherHitsValue2"); - - // increase hits number - assertEquals("higherHitsValue2", mapA2.get("key2")); - assertEquals("higherHitsValue2", mapA2.get("key2")); - } - - private void afterMergeHigherHitsMergePolicy() { - assertEquals("higherHitsValue1", mapA1.get("key1")); - assertEquals("higherHitsValue1", mapA2.get("key1")); - assertBackupEntryEqualsEventually("key1", "higherHitsValue1", backupMapA); - - assertEquals("higherHitsValue2", mapA1.get("key2")); - assertEquals("higherHitsValue2", mapA2.get("key2")); - assertBackupEntryEqualsEventually("key2", "higherHitsValue2", backupMapA); - - assertEquals(2, mapA1.size()); - assertEquals(2, mapA2.size()); - assertBackupSizeEventually(2, backupMapA); - } - - private void afterSplitLatestUpdateMergePolicy() { - mapA1.put("key1", "value1"); - - // prevent updating at the same time - sleepAtLeastMillis(1000); - - mapA2.put("key1", "LatestUpdatedValue1"); - mapA2.put("key2", "value2"); - - // prevent updating at the same time - sleepAtLeastMillis(1000); - - mapA1.put("key2", "LatestUpdatedValue2"); - } - - private void afterMergeLatestUpdateMergePolicy() { - assertEquals("LatestUpdatedValue1", mapA1.get("key1")); - assertEquals("LatestUpdatedValue1", mapA2.get("key1")); - assertBackupEntryEqualsEventually("key1", "LatestUpdatedValue1", backupMapA); - - assertEquals("LatestUpdatedValue2", mapA1.get("key2")); - assertEquals("LatestUpdatedValue2", mapA2.get("key2")); - assertBackupEntryEqualsEventually("key2", "LatestUpdatedValue2", backupMapA); - - assertEquals(2, mapA1.size()); - assertEquals(2, mapA2.size()); - assertBackupSizeEventually(2, backupMapA); - } - - private void afterSplitPassThroughMergePolicy() { - mapA1.put("key1", "value1"); - - mapA2.put("key1", "PassThroughValue1"); - mapA2.put("key2", "PassThroughValue2"); - - mapB2.put("key", "PutIfAbsentValue"); - } - - private void afterMergePassThroughMergePolicy() { - assertEquals("PassThroughValue1", mapA1.get("key1")); - assertEquals("PassThroughValue1", mapA2.get("key1")); - assertBackupEntryEqualsEventually("key1", "PassThroughValue1", backupMapA); - - assertEquals("PassThroughValue2", mapA1.get("key2")); - assertEquals("PassThroughValue2", mapA2.get("key2")); - assertBackupEntryEqualsEventually("key2", "PassThroughValue2", backupMapA); - - assertEquals(2, mapA1.size()); - assertEquals(2, mapA2.size()); - assertBackupSizeEventually(2, backupMapA); - - assertEquals("PutIfAbsentValue", mapB1.get("key")); - assertEquals("PutIfAbsentValue", mapB2.get("key")); - assertBackupEntryEqualsEventually("key", "PutIfAbsentValue", backupMapB); - - assertEquals(1, mapB1.size()); - assertEquals(1, mapB2.size()); - assertBackupSizeEventually(1, backupMapB); - } - - private void afterSplitPutIfAbsentMergePolicy() { - mapA1.put("key1", "PutIfAbsentValue1"); - - mapA2.put("key1", "value"); - mapA2.put("key2", "PutIfAbsentValue2"); - - mapB2.put("key", "PutIfAbsentValue"); - } - - private void afterMergePutIfAbsentMergePolicy() { - assertEquals("PutIfAbsentValue1", mapA1.get("key1")); - assertEquals("PutIfAbsentValue1", mapA2.get("key1")); - assertBackupEntryEqualsEventually("key1", "PutIfAbsentValue1", backupMapA); - - assertEquals("PutIfAbsentValue2", mapA1.get("key2")); - assertEquals("PutIfAbsentValue2", mapA2.get("key2")); - assertBackupEntryEqualsEventually("key2", "PutIfAbsentValue2", backupMapA); - - assertEquals(2, mapA1.size()); - assertEquals(2, mapA2.size()); - assertBackupSizeEventually(2, backupMapA); - - assertEquals("PutIfAbsentValue", mapB1.get("key")); - assertEquals("PutIfAbsentValue", mapB2.get("key")); - assertBackupEntryEqualsEventually("key", "PutIfAbsentValue", backupMapB); - - assertEquals(1, mapB1.size()); - assertEquals(1, mapB2.size()); - assertBackupSizeEventually(1, backupMapB); - } - - private void afterSplitCustomMergePolicy() { - mapA1.put("key", "value"); - mapA2.put("key", 1); - } - - private void afterMergeCustomMergePolicy() { - assertEquals(1, mapA1.get("key")); - assertEquals(1, mapA2.get("key")); - assertBackupEntryEqualsEventually("key", 1, backupMapA); - - assertEquals(1, mapA1.size()); - assertEquals(1, mapA2.size()); - assertBackupSizeEventually(1, backupMapA); - } - - protected static class CustomLegacyMergePolicy implements MapMergePolicy, DataSerializable { - - @Override - public Object merge(String mapName, EntryView mergingEntry, EntryView existingEntry) { - if (mergingEntry.getValue() instanceof Integer) { - return mergingEntry.getValue(); - } - return null; - } - - @Override - public void writeData(ObjectDataOutput out) { - } - - @Override - public void readData(ObjectDataInput in) { - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/MapSplitBrainStressTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/MapSplitBrainStressTest.java index d3a7ebbc24bb1..93d9aa79d15f3 100644 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/MapSplitBrainStressTest.java +++ b/hazelcast/src/test/java/com/hazelcast/map/merge/MapSplitBrainStressTest.java @@ -18,11 +18,12 @@ import com.hazelcast.config.Config; import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.map.IMap; import com.hazelcast.core.LifecycleEvent; import com.hazelcast.core.LifecycleListener; import com.hazelcast.logging.ILogger; import com.hazelcast.logging.Logger; +import com.hazelcast.map.IMap; +import com.hazelcast.spi.merge.PassThroughMergePolicy; import com.hazelcast.test.HazelcastSerialClassRunner; import com.hazelcast.test.SplitBrainTestSupport; import com.hazelcast.test.annotation.NightlyTest; @@ -69,7 +70,8 @@ public class MapSplitBrainStressTest extends SplitBrainTestSupport { protected Config config() { Config config = super.config(); config.getMapConfig(MAP_NAME_PREFIX + "*") - .setMergePolicy(MERGE_POLICY.getName()); + .getMergePolicyConfig() + .setPolicy(MERGE_POLICY.getName()); return config; } diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/MergePolicyProviderTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/MergePolicyProviderTest.java deleted file mode 100644 index 272316b2a5ebd..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/MergePolicyProviderTest.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.config.InvalidConfigurationException; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.HazelcastTestSupport; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class MergePolicyProviderTest extends HazelcastTestSupport { - - private MergePolicyProvider mergePolicyProvider; - - @Rule - public ExpectedException expected = ExpectedException.none(); - - @Before - public void given() { - mergePolicyProvider = new MergePolicyProvider(getNode(createHazelcastInstance()).getNodeEngine()); - } - - @Test - public void getMergePolicy_NotExistingMergePolicy() { - expected.expect(InvalidConfigurationException.class); - mergePolicyProvider.getMergePolicy("no such policy bro!"); - } - - @Test - public void getMergePolicy_NullPolicy() { - expected.expect(InvalidConfigurationException.class); - mergePolicyProvider.getMergePolicy(null); - } - - @Test - public void getMergePolicy_PutIfAbsentMapMergePolicy() { - assertMergePolicyCorrectlyInitialised("com.hazelcast.map.merge.PutIfAbsentMapMergePolicy"); - } - - @Test - public void getMergePolicy_LatestUpdateMapMergePolicy() { - assertMergePolicyCorrectlyInitialised("com.hazelcast.map.merge.LatestUpdateMapMergePolicy"); - } - - @Test - public void getMergePolicy_PassThroughMergePolicy() { - assertMergePolicyCorrectlyInitialised("com.hazelcast.map.merge.PassThroughMergePolicy"); - } - - @Test - public void getMergePolicy_HigherHitsMapMergePolicy() { - assertMergePolicyCorrectlyInitialised("com.hazelcast.map.merge.HigherHitsMapMergePolicy"); - } - - private void assertMergePolicyCorrectlyInitialised(String mergePolicyName) { - Object mergePolicy = mergePolicyProvider.getMergePolicy(mergePolicyName); - - assertNotNull(mergePolicy); - assertEquals(mergePolicyName, mergePolicy.getClass().getName()); - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/PassThroughMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/PassThroughMapMergePolicyTest.java deleted file mode 100644 index 7963aea238eed..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/PassThroughMapMergePolicyTest.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.core.EntryView; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class PassThroughMapMergePolicyTest { - - private static final String EXISTING = "EXISTING"; - private static final String MERGING = "MERGING"; - - protected MapMergePolicy policy; - - @Before - public void given() { - policy = new PassThroughMergePolicy(); - } - - @Test - public void merge_mergingNotNull() { - EntryView existing = entryWithGivenValue(EXISTING); - EntryView merging = entryWithGivenValue(MERGING); - - assertEquals(MERGING, policy.merge("map", merging, existing)); - } - - @Test - @SuppressWarnings("ConstantConditions") - public void merge_mergingNull() { - EntryView existing = entryWithGivenValue(EXISTING); - EntryView merging = null; - - assertEquals(EXISTING, policy.merge("map", merging, existing)); - } - - private EntryView entryWithGivenValue(String value) { - EntryView entryView = mock(EntryView.class); - try { - when(entryView.getValue()).thenReturn(value); - return entryView; - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/map/merge/PutIfAbsentMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/map/merge/PutIfAbsentMapMergePolicyTest.java deleted file mode 100644 index 24e0f073364aa..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/map/merge/PutIfAbsentMapMergePolicyTest.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.map.merge; - -import com.hazelcast.core.EntryView; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class PutIfAbsentMapMergePolicyTest { - - private static final String EXISTING = "EXISTING"; - private static final String MERGING = "MERGING"; - - protected MapMergePolicy policy; - - @Before - public void given() { - policy = new PutIfAbsentMapMergePolicy(); - } - - @Test - public void merge_existingValueAbsent() { - EntryView existing = entryWithGivenValue(null); - EntryView merging = entryWithGivenValue(MERGING); - - assertEquals(MERGING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_existingValuePresent() { - EntryView existing = entryWithGivenValue(EXISTING); - EntryView merging = entryWithGivenValue(MERGING); - - assertEquals(EXISTING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_bothValuesNull() { - EntryView existing = entryWithGivenValue(null); - EntryView merging = entryWithGivenValue(null); - - assertNull(policy.merge("map", merging, existing)); - } - - private EntryView entryWithGivenValue(String value) { - EntryView entryView = mock(EntryView.class); - try { - when(entryView.getValue()).thenReturn(value); - return entryView; - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/replicatedmap/CustomReplicatedMapMergePolicy.java b/hazelcast/src/test/java/com/hazelcast/replicatedmap/CustomReplicatedMapMergePolicy.java index 7d6f9f5a21c51..6a8b15cbcca2f 100644 --- a/hazelcast/src/test/java/com/hazelcast/replicatedmap/CustomReplicatedMapMergePolicy.java +++ b/hazelcast/src/test/java/com/hazelcast/replicatedmap/CustomReplicatedMapMergePolicy.java @@ -16,15 +16,31 @@ package com.hazelcast.replicatedmap; -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; -import com.hazelcast.replicatedmap.merge.ReplicatedMapMergePolicy; +import com.hazelcast.nio.ObjectDataInput; +import com.hazelcast.nio.ObjectDataOutput; +import com.hazelcast.spi.merge.MergingValue; +import com.hazelcast.spi.merge.SplitBrainMergePolicy; + +import java.io.IOException; + +public class CustomReplicatedMapMergePolicy + implements SplitBrainMergePolicy> { -public class CustomReplicatedMapMergePolicy implements ReplicatedMapMergePolicy { @Override - public Object merge(String mapName, ReplicatedMapEntryView mergingEntry, ReplicatedMapEntryView existingEntry) { - if (mergingEntry.getValue() instanceof Integer) { - return mergingEntry.getValue(); + public Integer merge(MergingValue mergingValue, MergingValue existingValue) { + if (mergingValue.getValue() instanceof Integer) { + return mergingValue.getValue(); } return null; } + + @Override + public void writeData(ObjectDataOutput out) throws IOException { + + } + + @Override + public void readData(ObjectDataInput in) throws IOException { + + } } diff --git a/hazelcast/src/test/java/com/hazelcast/replicatedmap/ReplicatedMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/replicatedmap/ReplicatedMapMergePolicyTest.java index d4777c33e310b..f5760400e3a0f 100644 --- a/hazelcast/src/test/java/com/hazelcast/replicatedmap/ReplicatedMapMergePolicyTest.java +++ b/hazelcast/src/test/java/com/hazelcast/replicatedmap/ReplicatedMapMergePolicyTest.java @@ -21,10 +21,10 @@ import com.hazelcast.core.HazelcastInstance; import com.hazelcast.core.LifecycleEvent; import com.hazelcast.core.LifecycleListener; -import com.hazelcast.replicatedmap.merge.HigherHitsMapMergePolicy; -import com.hazelcast.replicatedmap.merge.LatestUpdateMapMergePolicy; -import com.hazelcast.replicatedmap.merge.PassThroughMergePolicy; -import com.hazelcast.replicatedmap.merge.PutIfAbsentMapMergePolicy; +import com.hazelcast.spi.merge.HigherHitsMergePolicy; +import com.hazelcast.spi.merge.LatestUpdateMergePolicy; +import com.hazelcast.spi.merge.PassThroughMergePolicy; +import com.hazelcast.spi.merge.PutIfAbsentMergePolicy; import com.hazelcast.spi.properties.GroupProperty; import com.hazelcast.test.AssertTask; import com.hazelcast.test.HazelcastSerialParametersRunnerFactory; @@ -122,7 +122,7 @@ private Config newConfig(String mergePolicy, String mapName) { config.setProperty(GroupProperty.MERGE_NEXT_RUN_DELAY_SECONDS.getName(), "3"); config.getGroupConfig().setName(generateRandomString(10)); ReplicatedMapConfig replicatedMapConfig = config.getReplicatedMapConfig(mapName); - replicatedMapConfig.setMergePolicy(mergePolicy); + replicatedMapConfig.getMergePolicyConfig().setPolicy(mergePolicy); return config; } @@ -175,7 +175,7 @@ public Map populateMaps(ReplicatedMap map1, Repl @Override public String getMergePolicyClassName() { - return LatestUpdateMapMergePolicy.class.getName(); + return LatestUpdateMergePolicy.class.getName(); } @Override @@ -209,7 +209,7 @@ public Map populateMaps(ReplicatedMap map1, Repl @Override public String getMergePolicyClassName() { - return HigherHitsMapMergePolicy.class.getName(); + return HigherHitsMergePolicy.class.getName(); } @Override @@ -236,7 +236,7 @@ public Map populateMaps(ReplicatedMap map1, Repl @Override public String getMergePolicyClassName() { - return PutIfAbsentMapMergePolicy.class.getName(); + return PutIfAbsentMergePolicy.class.getName(); } @Override diff --git a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/AbstractReplicatedMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/AbstractReplicatedMapMergePolicyTest.java deleted file mode 100644 index 36788f9745940..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/AbstractReplicatedMapMergePolicyTest.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public abstract class AbstractReplicatedMapMergePolicyTest { - - private static final String EXISTING = "EXISTING"; - private static final String MERGING = "MERGING"; - - protected ReplicatedMapMergePolicy policy; - - @Test - public void merge_mergingWins() { - ReplicatedMapEntryView existing = entryWithGivenPropertyAndValue(1, EXISTING); - ReplicatedMapEntryView merging = entryWithGivenPropertyAndValue(333, MERGING); - - assertEquals(MERGING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_existingWins() { - ReplicatedMapEntryView existing = entryWithGivenPropertyAndValue(333, EXISTING); - ReplicatedMapEntryView merging = entryWithGivenPropertyAndValue(1, MERGING); - - assertEquals(EXISTING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_draw_mergingWins() { - ReplicatedMapEntryView existing = entryWithGivenPropertyAndValue(1, EXISTING); - ReplicatedMapEntryView merging = entryWithGivenPropertyAndValue(1, MERGING); - - assertEquals(MERGING, policy.merge("map", merging, existing)); - } - - private ReplicatedMapEntryView entryWithGivenPropertyAndValue(long testedProperty, String value) { - ReplicatedMapEntryView entryView = mock(ReplicatedMapEntryView.class); - try { - if (policy instanceof HigherHitsMapMergePolicy) { - when(entryView.getHits()).thenReturn(testedProperty); - } else if (policy instanceof LatestUpdateMapMergePolicy) { - when(entryView.getLastUpdateTime()).thenReturn(testedProperty); - } - when(entryView.getValue()).thenReturn(value); - return entryView; - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/HigherHitsReplicatedMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/HigherHitsReplicatedMapMergePolicyTest.java deleted file mode 100644 index 2f5c933fb1a6f..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/HigherHitsReplicatedMapMergePolicyTest.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class HigherHitsReplicatedMapMergePolicyTest extends AbstractReplicatedMapMergePolicyTest { - - @Before - public void given() { - policy = HigherHitsMapMergePolicy.INSTANCE; - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/LatestUpdateReplicatedMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/LatestUpdateReplicatedMapMergePolicyTest.java deleted file mode 100644 index 80936731c29a8..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/LatestUpdateReplicatedMapMergePolicyTest.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class LatestUpdateReplicatedMapMergePolicyTest extends AbstractReplicatedMapMergePolicyTest { - - @Before - public void given() { - policy = LatestUpdateMapMergePolicy.INSTANCE; - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/LegacyReplicatedMapSplitBrainTest.java b/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/LegacyReplicatedMapSplitBrainTest.java deleted file mode 100644 index 3d3b1ab144847..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/LegacyReplicatedMapSplitBrainTest.java +++ /dev/null @@ -1,235 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.config.Config; -import com.hazelcast.config.MergePolicyConfig; -import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.replicatedmap.ReplicatedMap; -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; -import com.hazelcast.test.AssertTask; -import com.hazelcast.test.HazelcastParallelParametersRunnerFactory; -import com.hazelcast.test.SplitBrainTestSupport; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; -import org.junit.runners.Parameterized.UseParametersRunnerFactory; - -import java.util.Collection; - -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -@RunWith(Parameterized.class) -@UseParametersRunnerFactory(HazelcastParallelParametersRunnerFactory.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class LegacyReplicatedMapSplitBrainTest extends SplitBrainTestSupport { - - @Parameters(name = "mergePolicy:{0}") - public static Collection parameters() { - return asList(new Object[]{ - LatestUpdateMapMergePolicy.class, - HigherHitsMapMergePolicy.class, - PutIfAbsentMapMergePolicy.class, - PassThroughMergePolicy.class, - CustomReplicatedMergePolicy.class, - }); - } - - @Parameter - public Class mergePolicyClass; - - private String replicatedMapName = randomMapName(); - private MergeLifecycleListener mergeLifecycleListener; - private ReplicatedMap replicatedMap1; - private ReplicatedMap replicatedMap2; - private String key1; - private String key2; - - @Override - protected Config config() { - MergePolicyConfig mergePolicyConfig = new MergePolicyConfig() - .setPolicy(mergePolicyClass.getName()) - .setBatchSize(10); - - Config config = super.config(); - config.getReplicatedMapConfig(replicatedMapName) - .setMergePolicyConfig(mergePolicyConfig); - return config; - } - - @Override - protected void onBeforeSplitBrainCreated(HazelcastInstance[] instances) { - warmUpPartitions(instances); - } - - @Override - protected void onAfterSplitBrainCreated(HazelcastInstance[] firstBrain, HazelcastInstance[] secondBrain) { - mergeLifecycleListener = new MergeLifecycleListener(secondBrain.length); - for (HazelcastInstance instance : secondBrain) { - instance.getLifecycleService().addLifecycleListener(mergeLifecycleListener); - } - - replicatedMap1 = firstBrain[0].getReplicatedMap(replicatedMapName); - replicatedMap2 = secondBrain[0].getReplicatedMap(replicatedMapName); - - // since the statistics are not replicated, we have to use keys on the same node like the proxy we are interacting with - String[] keys = generateKeysBelongingToSamePartitionsOwnedBy(firstBrain[0], 2); - key1 = keys[0]; - key2 = keys[1]; - - if (mergePolicyClass == LatestUpdateMapMergePolicy.class) { - afterSplitLatestUpdateMapMergePolicy(); - } else if (mergePolicyClass == HigherHitsMapMergePolicy.class) { - afterSplitHigherHitsMapMergePolicy(); - } else if (mergePolicyClass == PutIfAbsentMapMergePolicy.class) { - afterSplitPutIfAbsentMapMergePolicy(); - } else if (mergePolicyClass == PassThroughMergePolicy.class) { - afterSplitPassThroughMapMergePolicy(); - } else if (mergePolicyClass == CustomReplicatedMergePolicy.class) { - afterSplitCustomReplicatedMapMergePolicy(); - } else { - fail(); - } - } - - @Override - protected void onAfterSplitBrainHealed(HazelcastInstance[] instances) { - // wait until merge completes - mergeLifecycleListener.await(); - - AssertTask assertTask = new AssertTask() { - @Override - public void run() { - if (mergePolicyClass == LatestUpdateMapMergePolicy.class) { - afterMergeLatestUpdateMapMergePolicy(); - } else if (mergePolicyClass == HigherHitsMapMergePolicy.class) { - afterMergeHigherHitsMapMergePolicy(); - } else if (mergePolicyClass == PutIfAbsentMapMergePolicy.class) { - afterMergePutIfAbsentMapMergePolicy(); - } else if (mergePolicyClass == PassThroughMergePolicy.class) { - afterMergePassThroughMapMergePolicy(); - } else if (mergePolicyClass == CustomReplicatedMergePolicy.class) { - afterMergeCustomReplicatedMapMergePolicy(); - } else { - fail(); - } - } - }; - - // wait completion of migration tasks after lite member promotion - assertTrueEventually(assertTask); - } - - private void afterSplitLatestUpdateMapMergePolicy() { - replicatedMap1.put(key1, "value1"); - - // prevent updating at the same time - sleepAtLeastMillis(100); - - replicatedMap2.put(key1, "LatestUpdatedValue1"); - replicatedMap2.put(key2, "value2"); - - // prevent updating at the same time - sleepAtLeastMillis(100); - - replicatedMap1.put(key2, "LatestUpdatedValue2"); - } - - private void afterMergeLatestUpdateMapMergePolicy() { - assertEquals("LatestUpdatedValue1", replicatedMap1.get(key1)); - assertEquals("LatestUpdatedValue1", replicatedMap2.get(key1)); - - assertEquals("LatestUpdatedValue2", replicatedMap1.get(key2)); - assertEquals("LatestUpdatedValue2", replicatedMap2.get(key2)); - } - - private void afterSplitHigherHitsMapMergePolicy() { - replicatedMap1.put(key1, "higherHitsValue1"); - replicatedMap1.put(key2, "value2"); - - // increase hits number - assertEquals("higherHitsValue1", replicatedMap1.get(key1)); - assertEquals("higherHitsValue1", replicatedMap1.get(key1)); - - replicatedMap2.put(key1, "value1"); - replicatedMap2.put(key2, "higherHitsValue2"); - - // increase hits number - assertEquals("higherHitsValue2", replicatedMap2.get(key2)); - assertEquals("higherHitsValue2", replicatedMap2.get(key2)); - } - - private void afterMergeHigherHitsMapMergePolicy() { - assertEquals("higherHitsValue1", replicatedMap1.get(key1)); - assertEquals("higherHitsValue1", replicatedMap2.get(key1)); - - assertEquals("higherHitsValue2", replicatedMap1.get(key2)); - assertEquals("higherHitsValue2", replicatedMap2.get(key2)); - } - - private void afterSplitPutIfAbsentMapMergePolicy() { - replicatedMap1.put(key1, "PutIfAbsentValue1"); - - replicatedMap2.put(key1, "value1"); - replicatedMap2.put(key2, "PutIfAbsentValue2"); - } - - private void afterMergePutIfAbsentMapMergePolicy() { - assertEquals("PutIfAbsentValue1", replicatedMap1.get(key1)); - assertEquals("PutIfAbsentValue1", replicatedMap2.get(key1)); - - assertEquals("PutIfAbsentValue2", replicatedMap1.get(key2)); - assertEquals("PutIfAbsentValue2", replicatedMap2.get(key2)); - } - - private void afterSplitPassThroughMapMergePolicy() { - replicatedMap1.put(key1, "value"); - replicatedMap2.put(key1, "passThroughValue"); - } - - private void afterMergePassThroughMapMergePolicy() { - assertEquals("passThroughValue", replicatedMap1.get(key1)); - assertEquals("passThroughValue", replicatedMap2.get(key1)); - } - - private void afterSplitCustomReplicatedMapMergePolicy() { - replicatedMap1.put(key1, "value"); - replicatedMap2.put(key1, 1); - } - - private void afterMergeCustomReplicatedMapMergePolicy() { - assertEquals(1, replicatedMap1.get(key1)); - assertEquals(1, replicatedMap2.get(key1)); - } - - private static class CustomReplicatedMergePolicy implements ReplicatedMapMergePolicy { - - @Override - public Object merge(String replicatedMapName, ReplicatedMapEntryView mergingEntry, ReplicatedMapEntryView existingEntry) { - if (mergingEntry.getValue() instanceof Integer) { - return mergingEntry.getValue(); - } - return null; - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/MergePolicyProviderTest.java b/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/MergePolicyProviderTest.java deleted file mode 100644 index d230c13f53a7e..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/MergePolicyProviderTest.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.config.InvalidConfigurationException; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.HazelcastTestSupport; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class MergePolicyProviderTest extends HazelcastTestSupport { - - private MergePolicyProvider mergePolicyProvider; - - @Rule - public ExpectedException expected = ExpectedException.none(); - - @Before - public void given() { - mergePolicyProvider = new MergePolicyProvider(getNode(createHazelcastInstance()).getNodeEngine()); - } - - @Test - public void getMergePolicy_NotExistingMergePolicy() { - expected.expect(InvalidConfigurationException.class); - mergePolicyProvider.getMergePolicy("no such policy bro!"); - } - - @Test - public void getMergePolicy_NullPolicy() { - expected.expect(InvalidConfigurationException.class); - mergePolicyProvider.getMergePolicy(null); - } - - @Test - public void getMergePolicy_PutIfAbsentMapMergePolicy() { - assertMergePolicyCorrectlyInitialised("com.hazelcast.replicatedmap.merge.PutIfAbsentMapMergePolicy"); - } - - @Test - public void getMergePolicy_LatestUpdateMapMergePolicy() { - assertMergePolicyCorrectlyInitialised("com.hazelcast.replicatedmap.merge.LatestUpdateMapMergePolicy"); - } - - @Test - public void getMergePolicy_PassThroughMergePolicy() { - assertMergePolicyCorrectlyInitialised("com.hazelcast.replicatedmap.merge.PassThroughMergePolicy"); - } - - @Test - public void getMergePolicy_HigherHitsMapMergePolicy() { - assertMergePolicyCorrectlyInitialised("com.hazelcast.replicatedmap.merge.HigherHitsMapMergePolicy"); - } - - private void assertMergePolicyCorrectlyInitialised(String mergePolicyName) { - Object mergePolicy = mergePolicyProvider.getMergePolicy(mergePolicyName); - - assertNotNull(mergePolicy); - assertEquals(mergePolicyName, mergePolicy.getClass().getName()); - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/PassThroughMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/PassThroughMapMergePolicyTest.java deleted file mode 100644 index d70878c0eed6f..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/PassThroughMapMergePolicyTest.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class PassThroughMapMergePolicyTest { - - private static final String EXISTING = "EXISTING"; - private static final String MERGING = "MERGING"; - - protected ReplicatedMapMergePolicy policy; - - @Before - public void given() { - policy = PassThroughMergePolicy.INSTANCE; - } - - @Test - public void merge_mergingNotNull() { - ReplicatedMapEntryView existing = entryWithGivenValue(EXISTING); - ReplicatedMapEntryView merging = entryWithGivenValue(MERGING); - - assertEquals(MERGING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_mergingNull() { - ReplicatedMapEntryView existing = entryWithGivenValue(EXISTING); - - assertEquals(EXISTING, policy.merge("map", null, existing)); - } - - private ReplicatedMapEntryView entryWithGivenValue(String value) { - ReplicatedMapEntryView entryView = mock(ReplicatedMapEntryView.class); - try { - when(entryView.getValue()).thenReturn(value); - return entryView; - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/PutIfAbsentMapMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/PutIfAbsentMapMergePolicyTest.java deleted file mode 100644 index 9f92c5eba4cd1..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/replicatedmap/merge/PutIfAbsentMapMergePolicyTest.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.replicatedmap.merge; - -import com.hazelcast.replicatedmap.impl.record.ReplicatedMapEntryView; -import com.hazelcast.test.HazelcastParallelClassRunner; -import com.hazelcast.test.annotation.ParallelJVMTest; -import com.hazelcast.test.annotation.QuickTest; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -@RunWith(HazelcastParallelClassRunner.class) -@Category({QuickTest.class, ParallelJVMTest.class}) -public class PutIfAbsentMapMergePolicyTest { - - private static final String EXISTING = "EXISTING"; - private static final String MERGING = "MERGING"; - - protected ReplicatedMapMergePolicy policy; - - @Before - public void given() { - policy = PutIfAbsentMapMergePolicy.INSTANCE; - } - - @Test - public void merge_existingValueAbsent() { - ReplicatedMapEntryView existing = entryWithGivenValue(null); - ReplicatedMapEntryView merging = entryWithGivenValue(MERGING); - - assertEquals(MERGING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_existingValuePresent() { - ReplicatedMapEntryView existing = entryWithGivenValue(EXISTING); - ReplicatedMapEntryView merging = entryWithGivenValue(MERGING); - - assertEquals(EXISTING, policy.merge("map", merging, existing)); - } - - @Test - public void merge_bothValuesNull() { - ReplicatedMapEntryView existing = entryWithGivenValue(null); - ReplicatedMapEntryView merging = entryWithGivenValue(null); - - assertNull(policy.merge("map", merging, existing)); - } - - private ReplicatedMapEntryView entryWithGivenValue(String value) { - ReplicatedMapEntryView entryView = mock(ReplicatedMapEntryView.class); - try { - when(entryView.getValue()).thenReturn(value); - return entryView; - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/spi/merge/DiscardMergePolicyTest.java b/hazelcast/src/test/java/com/hazelcast/spi/merge/DiscardMergePolicyTest.java index 4ac386b6585ca..1ef0fa407ef0f 100644 --- a/hazelcast/src/test/java/com/hazelcast/spi/merge/DiscardMergePolicyTest.java +++ b/hazelcast/src/test/java/com/hazelcast/spi/merge/DiscardMergePolicyTest.java @@ -45,7 +45,7 @@ public class DiscardMergePolicyTest { @Before public void setup() { - mergePolicy = new DiscardMergePolicy(); + mergePolicy = new DiscardMergePolicy<>(); } @Test diff --git a/hazelcast/src/test/java/com/hazelcast/test/starter/constructor/MergePolicyProviderConstructor.java b/hazelcast/src/test/java/com/hazelcast/test/starter/constructor/MergePolicyProviderConstructor.java deleted file mode 100644 index 0b4eb84fa64dc..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/test/starter/constructor/MergePolicyProviderConstructor.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.test.starter.constructor; - -import com.hazelcast.test.starter.HazelcastStarterConstructor; -import com.hazelcast.test.starter.answer.NodeEngineAnswer; - -import java.lang.reflect.Constructor; - -import static com.hazelcast.test.starter.ReflectionUtils.getFieldValueReflectively; -import static org.mockito.Mockito.mock; - -@HazelcastStarterConstructor(classNames = {"com.hazelcast.spi.merge.SplitBrainMergePolicyProvider", - "com.hazelcast.cache.impl.merge.policy.CacheMergePolicyProvider"}) -public class MergePolicyProviderConstructor extends AbstractStarterObjectConstructor { - - public MergePolicyProviderConstructor(Class targetClass) { - super(targetClass); - } - - @Override - Object createNew0(Object delegate) throws Exception { - ClassLoader classloader = targetClass.getClassLoader(); - Class nodeEngineClass = classloader.loadClass("com.hazelcast.spi.NodeEngine"); - // obtain reference to constructor ...MergePolicyProvider(NodeEngine nodeEngine) - Constructor constructor = targetClass.getDeclaredConstructor(nodeEngineClass); - - Object nodeEngine = getFieldValueReflectively(delegate, "nodeEngine"); - Object[] args = new Object[]{mock(nodeEngineClass, new NodeEngineAnswer(nodeEngine))}; - - return constructor.newInstance(args); - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/test/starter/constructor/test/MergePolicyProviderConstructorTest.java b/hazelcast/src/test/java/com/hazelcast/test/starter/constructor/test/MergePolicyProviderConstructorTest.java deleted file mode 100644 index 30bd44f5535ae..0000000000000 --- a/hazelcast/src/test/java/com/hazelcast/test/starter/constructor/test/MergePolicyProviderConstructorTest.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Copyright (c) 2008-2019, Hazelcast, Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.hazelcast.test.starter.constructor.test; - -import com.hazelcast.cache.impl.CacheService; -import com.hazelcast.cache.impl.merge.policy.CacheMergePolicyProvider; -import com.hazelcast.config.Config; -import com.hazelcast.config.InvalidConfigurationException; -import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.spi.NodeEngine; -import com.hazelcast.test.HazelcastSerialClassRunner; -import com.hazelcast.test.HazelcastTestSupport; -import com.hazelcast.test.annotation.SlowTest; -import com.hazelcast.test.starter.HazelcastStarter; -import com.hazelcast.test.starter.constructor.MergePolicyProviderConstructor; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import static com.hazelcast.test.starter.HazelcastStarterUtils.assertInstanceOfByClassName; -import static com.hazelcast.test.starter.ReflectionUtils.getDelegateFromMock; -import static com.hazelcast.test.starter.ReflectionUtils.getFieldValueReflectively; -import static org.junit.Assert.fail; - -@RunWith(HazelcastSerialClassRunner.class) -@Category(SlowTest.class) -public class MergePolicyProviderConstructorTest extends HazelcastTestSupport { - - private HazelcastInstance hz; - - @Before - public void setUp() { - Config config = new Config(); - hz = HazelcastStarter.newHazelcastInstance("3.10", config, false); - } - - @After - public void tearDown() { - hz.shutdown(); - } - - @Test - public void testConstructor() throws Exception { - //this is testing the Hazelcast Started can create an instances of merge policies - - NodeEngine nodeEngine = getNodeEngineImpl(hz); - CacheService service = nodeEngine.getService(CacheService.SERVICE_NAME); - CacheMergePolicyProvider mergePolicyProvider = service.getMergePolicyProvider(); - - Object delegate = getDelegateFromMock(service); - Object mergePolicyProviderProxy = getFieldValueReflectively(delegate, "mergePolicyProvider"); - - MergePolicyProviderConstructor constructor = new MergePolicyProviderConstructor(CacheMergePolicyProvider.class); - CacheMergePolicyProvider clonedMergePolicyProvider - = (CacheMergePolicyProvider) constructor.createNew(mergePolicyProviderProxy); - - // invalid merge policy - assertInvalidCacheMergePolicy(mergePolicyProvider); - assertInvalidCacheMergePolicy(clonedMergePolicyProvider); - // legacy merge policy - assertCacheMergePolicy(mergePolicyProvider, "com.hazelcast.cache.merge.PutIfAbsentCacheMergePolicy"); - assertCacheMergePolicy(clonedMergePolicyProvider, "com.hazelcast.cache.merge.PutIfAbsentCacheMergePolicy"); - // unified merge policy - assertCacheMergePolicy(mergePolicyProvider, "com.hazelcast.spi.merge.PutIfAbsentMergePolicy"); - assertCacheMergePolicy(clonedMergePolicyProvider, "com.hazelcast.spi.merge.PutIfAbsentMergePolicy"); - } - - private static void assertInvalidCacheMergePolicy(CacheMergePolicyProvider mergePolicyProvider) { - try { - mergePolicyProvider.getMergePolicy("invalid"); - fail("Expected: InvalidConfigurationException: Invalid cache merge policy: invalid"); - } catch (InvalidConfigurationException expected) { - } - } - - private static void assertCacheMergePolicy(CacheMergePolicyProvider mergePolicyProvider, String mergePolicyClassname) { - Object mergePolicy = mergePolicyProvider.getMergePolicy(mergePolicyClassname); - assertInstanceOfByClassName(mergePolicyClassname, mergePolicy); - } -} diff --git a/hazelcast/src/test/java/com/hazelcast/wan/impl/WanConfigurationTest.java b/hazelcast/src/test/java/com/hazelcast/wan/impl/WanConfigurationTest.java index c7f30bb75d0ff..bb0f0d66b4c50 100644 --- a/hazelcast/src/test/java/com/hazelcast/wan/impl/WanConfigurationTest.java +++ b/hazelcast/src/test/java/com/hazelcast/wan/impl/WanConfigurationTest.java @@ -26,7 +26,7 @@ import com.hazelcast.map.impl.MapService; import com.hazelcast.map.impl.MapServiceContext; import com.hazelcast.map.impl.proxy.MapProxyImpl; -import com.hazelcast.map.merge.PassThroughMergePolicy; +import com.hazelcast.spi.merge.PassThroughMergePolicy; import com.hazelcast.test.HazelcastParallelClassRunner; import com.hazelcast.test.HazelcastTestSupport; import com.hazelcast.test.TestHazelcastInstanceFactory; diff --git a/hazelcast/src/test/java/com/hazelcast/wan/impl/WanReplicationTest.java b/hazelcast/src/test/java/com/hazelcast/wan/impl/WanReplicationTest.java index 466f60dc5bd7c..74af9d2c16245 100644 --- a/hazelcast/src/test/java/com/hazelcast/wan/impl/WanReplicationTest.java +++ b/hazelcast/src/test/java/com/hazelcast/wan/impl/WanReplicationTest.java @@ -22,21 +22,21 @@ import com.hazelcast.config.WanReplicationConfig; import com.hazelcast.config.WanReplicationRef; import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.map.IMap; import com.hazelcast.internal.partition.InternalPartitionService; import com.hazelcast.internal.serialization.InternalSerializationService; import com.hazelcast.map.EntryProcessor; +import com.hazelcast.map.IMap; import com.hazelcast.map.impl.MapService; import com.hazelcast.map.impl.MapServiceContext; import com.hazelcast.map.impl.SimpleEntryView; import com.hazelcast.map.impl.operation.MapOperation; import com.hazelcast.map.impl.operation.MapOperationProvider; import com.hazelcast.map.impl.proxy.MapProxyImpl; -import com.hazelcast.map.merge.PassThroughMergePolicy; import com.hazelcast.nio.serialization.Data; -import com.hazelcast.spi.impl.operationservice.OperationFactory; import com.hazelcast.spi.impl.NodeEngineImpl; +import com.hazelcast.spi.impl.operationservice.OperationFactory; import com.hazelcast.spi.impl.operationservice.impl.OperationServiceImpl; +import com.hazelcast.spi.merge.PassThroughMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergePolicy; import com.hazelcast.spi.merge.SplitBrainMergeTypes.MapMergeTypes; import com.hazelcast.spi.serialization.SerializationService; @@ -219,47 +219,25 @@ public void programmaticImplCreationTest() { assertEquals(dummyWanReplication, getWanReplicationImpl(instance1)); } - @Test - @SuppressWarnings("ConstantConditions") - public void mergeOperationGeneratesWanReplicationEvent_withLegacyMergePolicy() { - boolean enableWANReplicationEvent = true; - boolean useLegacyMergePolicy = true; - runMergeOpForWAN(enableWANReplicationEvent, useLegacyMergePolicy); - - assertTotalQueueSize(1); - } - @Test @SuppressWarnings("ConstantConditions") public void mergeOperationGeneratesWanReplicationEvent() { boolean enableWANReplicationEvent = true; - boolean useLegacyMergePolicy = false; - runMergeOpForWAN(enableWANReplicationEvent, useLegacyMergePolicy); + runMergeOpForWAN(enableWANReplicationEvent); assertTotalQueueSize(1); } - @Test - @SuppressWarnings("ConstantConditions") - public void mergeOperationDoesNotGenerateWanReplicationEventWhenDisabled_withLegacyMergePolicy() { - boolean enableWANReplicationEvent = false; - boolean useLegacyMergePolicy = true; - runMergeOpForWAN(enableWANReplicationEvent, useLegacyMergePolicy); - - assertTrueAllTheTime(() -> assertTotalQueueSize(0), 3); - } - @Test @SuppressWarnings("ConstantConditions") public void mergeOperationDoesNotGenerateWanReplicationEventWhenDisabled() { boolean enableWANReplicationEvent = false; - boolean useLegacyMergePolicy = false; - runMergeOpForWAN(enableWANReplicationEvent, useLegacyMergePolicy); + runMergeOpForWAN(enableWANReplicationEvent); assertTrueAllTheTime(() -> assertTotalQueueSize(0), 3); } - private void runMergeOpForWAN(boolean enableWANReplicationEvent, boolean useLegacyMergePolicy) { + private void runMergeOpForWAN(boolean enableWANReplicationEvent) { // init hazelcast instances String mapName = "merge_operation_generates_wan_replication_event"; initInstancesAndMap(mapName); @@ -278,15 +256,11 @@ private void runMergeOpForWAN(boolean enableWANReplicationEvent, boolean useLega Data data = serializationService.toData(1); MapOperation op; SimpleEntryView entryView = new SimpleEntryView().withKey(data).withValue(data); - if (useLegacyMergePolicy) { - op = operationProvider.createLegacyMergeOperation(mapName, entryView, new PassThroughMergePolicy(), - !enableWANReplicationEvent); - } else { - MapMergeTypes mergingEntry = createMergingEntry(serializationService, entryView); - SplitBrainMergePolicy mergePolicy - = new com.hazelcast.spi.merge.PassThroughMergePolicy<>(); - op = operationProvider.createMergeOperation(mapName, mergingEntry, mergePolicy, !enableWANReplicationEvent); - } + + MapMergeTypes mergingEntry = createMergingEntry(serializationService, entryView); + SplitBrainMergePolicy mergePolicy + = new com.hazelcast.spi.merge.PassThroughMergePolicy<>(); + op = operationProvider.createMergeOperation(mapName, mergingEntry, mergePolicy, !enableWANReplicationEvent); operationService.createInvocationBuilder(MapService.SERVICE_NAME, op, partitionService.getPartitionId(data)).invoke(); }