Skip to content

Commit

Permalink
expose pojo caching in consumer
Browse files Browse the repository at this point in the history
  • Loading branch information
workeatsleep committed Mar 25, 2023
1 parent 877dd52 commit fc9b9c0
Show file tree
Hide file tree
Showing 27 changed files with 1,053 additions and 9 deletions.
3 changes: 3 additions & 0 deletions config/checkstyle/checkstyle.xml
Original file line number Diff line number Diff line change
Expand Up @@ -17,4 +17,7 @@
<property name="separateLineBetweenGroups" value="true"/>
</module>
</module>
<module name="SuppressionFilter">
<property name="file" value="${config_loc}/suppression.xml"/>
</module>
</module>
32 changes: 32 additions & 0 deletions config/checkstyle/suppression.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
<!--
~ Licensed to the Apache Software Foundation (ASF) under one or more
~ contributor license agreements. See the NOTICE file distributed with
~ this work for additional information regarding copyright ownership.
~ The ASF licenses this file to You under the Apache License, Version 2.0
~ (the "License"); you may not use this file except in compliance with
~ the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->

<!DOCTYPE suppressions PUBLIC
"-//Puppy Crawl//DTD Suppressions 1.1//EN"
"http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">

<!--
This file contains suppression rules for Checkstyle checks.
Ideally only files that cannot be modified (e.g. third-party code)
should be added here. All other violations should be fixed.
-->

<suppressions>
<suppress checks=".*" files="[\\/]generated[\\/]"/>
</suppressions>
Original file line number Diff line number Diff line change
Expand Up @@ -175,6 +175,7 @@ public String generate() {

builder.append(" }\n\n");

builder.append("/** Objects are no longer accessible after this method is called, do not call this method unless the object will not be used anymore. */ \n");

builder.append(" public void detachCaches() {\n");
for(HollowSchema schema : schemaList) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1102,13 +1102,20 @@ public B withRefreshListeners(HollowConsumer.RefreshListener... refreshListeners
* @return this builder
* @throws IllegalArgumentException if provided API class is {@code HollowAPI} instead of a subclass
*/
public B withGeneratedAPIClass(Class<? extends HollowAPI> generatedAPIClass) {
public B withGeneratedAPIClass(Class<? extends HollowAPI> generatedAPIClass, String cachedType, String... additionalCachedTypes) {
if (HollowAPI.class.equals(generatedAPIClass))
throw new IllegalArgumentException("must provide a code generated API class");
this.apiFactory = new HollowAPIFactory.ForGeneratedAPI<>(generatedAPIClass);
String[] types = new String[additionalCachedTypes.length + 1];
types[0] = cachedType;
System.arraycopy(additionalCachedTypes, 0, types, 1, additionalCachedTypes.length);
this.apiFactory = new HollowAPIFactory.ForGeneratedAPI<>(generatedAPIClass, types);
return (B)this;
}

public B withGeneratedAPIClass(Class<? extends HollowAPI> generatedAPIClass) {
return withGeneratedAPIClass(generatedAPIClass, null);
}

/**
* <p>Specifies a filter config. Clears any type filter specified by {@code withTypeFilter(...)}.</p>
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,11 +36,11 @@
*/
public class HollowObjectCacheProvider<T> extends HollowObjectProvider<T> implements HollowTypeStateListener {
private static final Logger log = Logger.getLogger(HollowObjectCacheProvider.class.getName());
private final List<T> cachedItems;
private volatile List<T> cachedItems;

private HollowFactory<T> factory;
private HollowTypeAPI typeAPI;
private HollowTypeReadState typeReadState;
private volatile HollowFactory<T> factory;
private volatile HollowTypeAPI typeAPI;
private volatile HollowTypeReadState typeReadState;

public HollowObjectCacheProvider(HollowTypeDataAccess typeDataAccess, HollowTypeAPI typeAPI, HollowFactory<T> factory) {
this(typeDataAccess, typeAPI, factory, null);
Expand Down Expand Up @@ -84,11 +84,18 @@ public HollowObjectCacheProvider(HollowTypeDataAccess typeDataAccess, HollowType

@Override
public T getHollowObject(int ordinal) {
return cachedItems.get(ordinal);
List<T> refCachedItems = cachedItems; // SNAP: object cache
if (refCachedItems == null) {
throw new IllegalStateException("");
}
if (refCachedItems.size() <= ordinal) {
throw new IllegalStateException("");
}
return refCachedItems.get(ordinal);
}

public void detach() {
cachedItems.clear();
cachedItems = null;
factory = null;
typeAPI = null;
typeReadState = null;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
package com.netflix.hollow.api.consumer;

import com.netflix.hollow.test.HollowWriteStateEngineBuilder;
import com.netflix.hollow.test.consumer.TestBlobRetriever;
import com.netflix.hollow.test.consumer.TestHollowConsumer;
import com.netflix.hollow.test.generated.MovieAPI;
import com.netflix.hollow.test.model.Movie;
import java.io.IOException;
import org.junit.Assert;
import org.junit.Test;


public class HollowConsumerBuilderTest {
@Test
public void testCachedTypes() throws IOException {
TestHollowConsumer consumer = new TestHollowConsumer.Builder()
.withBlobRetriever(new TestBlobRetriever())
.withGeneratedAPIClass(MovieAPI.class, "Movie")
.build();
testConsumerCache(true, false, consumer);
}

@Test(expected = IllegalStateException.class)
public void testCachedTypesDetached() throws IOException {
TestHollowConsumer consumer = new TestHollowConsumer.Builder()
.withBlobRetriever(new TestBlobRetriever())
.withGeneratedAPIClass(MovieAPI.class, "Movie")
.build();
testConsumerCache(true, true, consumer);
}

@Test
public void testNoCacheConsumer() throws IOException {
TestHollowConsumer consumer = new TestHollowConsumer.Builder()
.withBlobRetriever(new TestBlobRetriever())
.withGeneratedAPIClass(MovieAPI.class)
.build();
testConsumerCache(false, false, consumer);
}

private void testConsumerCache(boolean hasCache, boolean detachCache, TestHollowConsumer consumer) throws IOException {
consumer.addSnapshot(1l, new HollowWriteStateEngineBuilder().add(
new Movie(1, "test movie 1", 2023)).build());

consumer.triggerRefreshTo(1l);

consumer.addDelta(1l, 2l, new HollowWriteStateEngineBuilder().add(
new Movie(1, "test movie 1", 2023),
new Movie(2, "test movie 2", 2023)).build());

consumer.triggerRefreshTo(2l);

Assert.assertTrue(consumer.getCurrentVersionId() == 2l);

MovieAPI movieAPI = (MovieAPI) consumer.getAPI();
if (detachCache) {
movieAPI.detachCaches();
}
com.netflix.hollow.test.generated.Movie movie1 = movieAPI.getMovie(0);
com.netflix.hollow.test.generated.Movie movie2 = movieAPI.getMovie(0);

if (hasCache) {
Assert.assertTrue(movie1 == movie2);
} else {
Assert.assertFalse(movie1 == movie2);
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ public void notification_afterDetaching() {
// filled with null
subject.get().getHollowObject(0);
fail("expected exception to be thrown");
} catch (IndexOutOfBoundsException expected) {}
} catch (IllegalStateException expected) {}
}

private void prepopulate(TypeA...population) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
package com.netflix.hollow.test.generated;

import com.netflix.hollow.api.consumer.HollowConsumer;
import com.netflix.hollow.api.objects.HollowObject;
import com.netflix.hollow.core.schema.HollowObjectSchema;
import com.netflix.hollow.core.write.objectmapper.HollowTypeName;

@SuppressWarnings("all")
@HollowTypeName(name="String")
public class HString extends HollowObject {

public HString(StringDelegate delegate, int ordinal) {
super(delegate, ordinal);
}

public String getValue() {
return delegate().getValue(ordinal);
}

public boolean isValueEqual(String testValue) {
return delegate().isValueEqual(ordinal, testValue);
}

public MovieAPI api() {
return typeApi().getAPI();
}

public StringTypeAPI typeApi() {
return delegate().getTypeAPI();
}

protected StringDelegate delegate() {
return (StringDelegate)delegate;
}

}
68 changes: 68 additions & 0 deletions hollow/src/test/java/com/netflix/hollow/test/generated/Movie.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
package com.netflix.hollow.test.generated;

import com.netflix.hollow.api.consumer.HollowConsumer;
import com.netflix.hollow.api.consumer.index.UniqueKeyIndex;
import com.netflix.hollow.api.objects.HollowObject;
import com.netflix.hollow.core.schema.HollowObjectSchema;

@SuppressWarnings("all")
public class Movie extends HollowObject {

public Movie(MovieDelegate delegate, int ordinal) {
super(delegate, ordinal);
}

public long getId() {
return delegate().getId(ordinal);
}

public Long getIdBoxed() {
return delegate().getIdBoxed(ordinal);
}

public HString getTitle() {
int refOrdinal = delegate().getTitleOrdinal(ordinal);
if(refOrdinal == -1)
return null;
return api().getHString(refOrdinal);
}

public int getYear() {
return delegate().getYear(ordinal);
}

public Integer getYearBoxed() {
return delegate().getYearBoxed(ordinal);
}

public MovieAPI api() {
return typeApi().getAPI();
}

public MovieTypeAPI typeApi() {
return delegate().getTypeAPI();
}

protected MovieDelegate delegate() {
return (MovieDelegate)delegate;
}

/**
* Creates a unique key index for {@code Movie} that has a primary key.
* The primary key is represented by the type {@code long}.
* <p>
* By default the unique key index will not track updates to the {@code consumer} and thus
* any changes will not be reflected in matched results. To track updates the index must be
* {@link HollowConsumer#addRefreshListener(HollowConsumer.RefreshListener) registered}
* with the {@code consumer}
*
* @param consumer the consumer
* @return the unique key index
*/
public static UniqueKeyIndex<Movie, Long> uniqueIndex(HollowConsumer consumer) {
return UniqueKeyIndex.from(consumer, Movie.class)
.bindToPrimaryKey()
.usingPath("id", long.class);
}

}
Loading

0 comments on commit fc9b9c0

Please sign in to comment.