Skip to content

Commit

Permalink
feat(pipeline): add a pipelineNameFilter query param to the /pipeline…
Browse files Browse the repository at this point in the history
…s/{application} endpoint (#1504)

* feat(pipelineController): add a pipelineNameFilter query param to the /pipelines/{application} endpoint

This adds a pipelineNameFilter query parameter to the /pipelines/{application} endpoint. If pipelineNameFilter is present, the endpoint will return a list of pipelines whose pipeline name contains the pipelineNameFilter.

This change is necessary to enable some optimizations in the front end - namely, filtering pipelines on the backend, instead of always querying for every pipeline in an application, and filtering the list in the front end.

* feat(pipeline): check for null pipeline name and log error

* fix(test/pipeline): updating tests and fixing getPipelinesByApplication implementation.

* fix(pipeline): make the comparison case insensitive.

* refactor(tests): convert groovy tests to java

* fix(tests): address minor issues

---------

Co-authored-by: Richard Timpson <[email protected]>
  • Loading branch information
kirangodishala and Richard Timpson authored Nov 6, 2024
1 parent e5f323d commit 51f2d15
Show file tree
Hide file tree
Showing 11 changed files with 323 additions and 46 deletions.
7 changes: 7 additions & 0 deletions front50-core/front50-core.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -41,4 +41,11 @@ dependencies {
implementation "io.spinnaker.kork:kork-moniker"

implementation project(":front50-api")

// this is a circular dependency, but given that it's only for tests
// it's probably ok to keep. The alternative is to move the PipelineDAO test classes
// in front50-test to front50-common, but front50-test seems like a better place.
testImplementation project(":front50-test")
testImplementation "io.spinnaker.kork:kork-sql-test"

}
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,15 @@
import java.util.Collection;
import java.util.UUID;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.util.Assert;
import org.springframework.util.ObjectUtils;
import rx.Scheduler;

public class DefaultPipelineDAO extends StorageServiceSupport<Pipeline> implements PipelineDAO {
private static final Logger log = LoggerFactory.getLogger(DefaultPipelineDAO.class);

public DefaultPipelineDAO(
StorageService service,
Scheduler scheduler,
Expand Down Expand Up @@ -72,11 +77,45 @@ public Collection<Pipeline> getPipelinesByApplication(String application) {

@Override
public Collection<Pipeline> getPipelinesByApplication(String application, boolean refresh) {
return getPipelinesByApplication(application, null, refresh);
}

@Override
public Collection<Pipeline> getPipelinesByApplication(
String application, String pipelineNameFilter, boolean refresh) {
return all(refresh).stream()
.filter(
pipeline ->
pipeline.getApplication() != null
&& pipeline.getApplication().equalsIgnoreCase(application))
pipeline -> {
/*
There's a sneaky bug where some application names are null. It's hard to find,
so for debugging purposes, we ALWAYS want to check for null pipeline names.
*/
if (pipeline.getName() == null) {
log.error(
"Pipeline with (id={}, app={}, type={}, lastModified={}) does not have a name.",
pipeline.getId(),
pipeline.getApplication(),
pipeline.getType(),
pipeline.getLastModified());
}

if (pipeline.getApplication() == null
|| !pipeline.getApplication().equalsIgnoreCase(application)) {
return false;
}

/*
if the pipeline name filter is empty, we want to treat it as if it doesn't exist.
if isEmpty returns true, the statement will short circuit and return true,
which effectively means we don't use the filter at all.
*/
return ObjectUtils.isEmpty(pipelineNameFilter)
|| pipeline.getName() != null
&& pipeline
.getName()
.toLowerCase()
.contains(pipelineNameFilter.toLowerCase());
})
.collect(Collectors.toList());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,5 +11,8 @@ public interface PipelineDAO extends ItemDAO<Pipeline> {

Collection<Pipeline> getPipelinesByApplication(String application, boolean refresh);

Collection<Pipeline> getPipelinesByApplication(
String application, String pipelineNameFilter, boolean refresh);

Pipeline getPipelineByName(String application, String pipelineName, boolean refresh);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
/*
* Copyright 2023 Salesforce, Inc.
*
* 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.netflix.spinnaker.front50.pipeline;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.Mockito.doReturn;

import com.netflix.spinnaker.front50.api.model.pipeline.Pipeline;
import com.netflix.spinnaker.front50.model.pipeline.DefaultPipelineDAO;
import com.netflix.spinnaker.kork.sql.test.SqlTestUtil;
import java.util.Collection;
import java.util.List;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource;
import org.mockito.Mockito;

public abstract class DefaultPipelineDAOTest extends PipelineDAOSpec<DefaultPipelineDAO> {

protected DefaultPipelineDAO pipelineDAO;

@Override
public DefaultPipelineDAO getInstance() {
return getDefaultPipelineDAO();
}

public abstract DefaultPipelineDAO getDefaultPipelineDAO();

@BeforeEach
public void setup() {
this.pipelineDAO = Mockito.spy(getDefaultPipelineDAO());
}

@ParameterizedTest
@CsvSource({
"'app', 'pipelineNameA', 'NameA', 'pipelineNameA'",
"'app', 'pipelineNameA', , 'pipelineNameA'",
"'app', , , "
})
public void shouldReturnCorrectPipelinesWhenRequestingPipelinesByApplicationWithNameFilter(
String applicationName,
String pipelineName,
String pipelineNameFilter,
String expectedPipelineName) {

Pipeline pipeline = new Pipeline();
pipeline.setId("0");
pipeline.setApplication(applicationName);
pipeline.setName(pipelineName);

doReturn(List.of(pipeline)).when(pipelineDAO).all(anyBoolean());

Collection<Pipeline> pipelines =
pipelineDAO.getPipelinesByApplication("app", pipelineNameFilter, true);

Pipeline resultPipeline = pipelines.iterator().next();
assertEquals(resultPipeline.getName(), expectedPipelineName);
assertEquals(resultPipeline.getApplication(), "app");
}

@ParameterizedTest
@CsvSource({
"'app', , 'NameA'",
"'bad', 'pipelineNameA', 'NameA'",
"'bad', , 'NameA'",
"'bad', 'pipelineNameA', ",
"'bad', , "
})
public void shouldReturnNoPipelinesWhenRequestingPipelinesByApplicationWithNameFilter(
String applicationName, String pipelineName, String pipelineNameFilter) {

Pipeline pipeline = new Pipeline();
pipeline.setId("0");
pipeline.setApplication(applicationName);
pipeline.setName(pipelineName);

doReturn(List.of(pipeline)).when(pipelineDAO).all(true);

Collection<Pipeline> pipelines =
pipelineDAO.getPipelinesByApplication("app", pipelineNameFilter, true);

assertEquals(0, pipelines.size());
}
}

class SqlDefaultPipelineDAOTest extends DefaultPipelineDAOTest {

private SqlTestUtil.TestDatabase database = SqlTestUtil.initTcMysqlDatabase();

@AfterEach
public void cleanup() {
if (database != null) {
SqlTestUtil.cleanupDb(database.context);
database.close();
}
}

@Override
public DefaultPipelineDAO getDefaultPipelineDAO() {
return SqlPipelineDAOTestConfiguration.createPipelineDAO(database);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import org.springframework.data.redis.core.Cursor
import org.springframework.data.redis.core.RedisTemplate
import org.springframework.data.redis.core.ScanOptions
import org.springframework.util.Assert
import org.springframework.util.ObjectUtils

class RedisPipelineDAO implements PipelineDAO {

Expand All @@ -39,8 +40,17 @@ class RedisPipelineDAO implements PipelineDAO {

@Override
Collection<Pipeline> getPipelinesByApplication(String application, boolean refresh = true) {
return getPipelinesByApplication(application, null, refresh)
}

@Override
Collection<Pipeline> getPipelinesByApplication(String application, String pipelineNameFilter, boolean refresh = true) {
all(refresh).findAll {
it.application == application
/* if the pipeline name filter is empty, we want to treat it as if it doesn't exist
if isEmpty returns true, the statement will short circuit and return true,
which effectively means we don't use the filter at all. */
it.getApplication() == application &&
(ObjectUtils.isEmpty(pipelineNameFilter) || it.getName().toLowerCase().contains(pipelineNameFilter.toLowerCase()))
}
}

Expand Down
3 changes: 3 additions & 0 deletions front50-test/front50-test.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,13 @@ dependencies {
implementation project(":front50-api")
implementation project(":front50-core")
implementation project(":front50-s3")
implementation project(":front50-sql")

implementation "org.apache.groovy:groovy"
implementation "org.spockframework:spock-core"
implementation "com.amazonaws:aws-java-sdk-s3"
implementation "io.spinnaker.kork:kork-sql-test"
implementation "io.reactivex:rxjava"
}

tasks.compileGroovy.enabled = true
Original file line number Diff line number Diff line change
Expand Up @@ -41,4 +41,36 @@ abstract class PipelineDAOSpec<T extends PipelineDAO> extends Specification {
thrown IllegalArgumentException
}

def "filters pipelines by pipeline name"() {
when:
instance.create("0", new Pipeline(application: "foo", name: "pipelineName1"))
for (int i = 1; i < 10; i++) {
def name = i % 2 == 0 ? "pipelineNameA" + i : "pipelineNameB" + i;
instance.create(i.toString(), new Pipeline(application: "foo", name: name))
}
def filteredPipelines = instance.getPipelinesByApplication("foo", "NameA", true);

then:
// the pipelines are not guaranteed to be in order of insertion
pipelinesContainName(filteredPipelines, "pipelineNameA2")
pipelinesContainName(filteredPipelines, "pipelineNameA4")
pipelinesContainName(filteredPipelines, "pipelineNameA6")
pipelinesContainName(filteredPipelines, "pipelineNameA8")
filteredPipelines.size() == 4
}

def "filters pipelines by pipeline name case insensitive"() {
when:
instance.create("0", new Pipeline(application: "foo", name: "PipElinenamea"))
def filteredPipelines = instance.getPipelinesByApplication("foo", "NameA", true);

then:
filteredPipelines[0].getName() == "PipElinenamea"
filteredPipelines.size() == 1
}

private static boolean pipelinesContainName(Collection<Pipeline> pipelines, String name) {
return pipelines.stream().filter {pipeline -> pipeline.getName() == name}.findAny().isPresent();
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* Copyright 2023 Salesforce, Inc.
*
* 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.netflix.spinnaker.front50.pipeline;

import com.fasterxml.jackson.databind.ObjectMapper;
import com.netflix.spectator.api.NoopRegistry;
import com.netflix.spinnaker.config.Front50SqlProperties;
import com.netflix.spinnaker.front50.config.StorageServiceConfigurationProperties;
import com.netflix.spinnaker.front50.model.DefaultObjectKeyLoader;
import com.netflix.spinnaker.front50.model.SqlStorageService;
import com.netflix.spinnaker.front50.model.pipeline.DefaultPipelineDAO;
import com.netflix.spinnaker.kork.sql.config.SqlRetryProperties;
import com.netflix.spinnaker.kork.sql.test.SqlTestUtil;
import io.github.resilience4j.circuitbreaker.CircuitBreakerRegistry;
import java.time.Clock;
import java.util.concurrent.Executors;
import rx.Scheduler;
import rx.schedulers.Schedulers;

public class SqlPipelineDAOTestConfiguration {

public static DefaultPipelineDAO createPipelineDAO(SqlTestUtil.TestDatabase database) {
Scheduler scheduler = Schedulers.from(Executors.newFixedThreadPool(1));

StorageServiceConfigurationProperties.PerObjectType pipelineDAOConfigProperties =
new StorageServiceConfigurationProperties().getPipeline();

SqlStorageService storageService =
new SqlStorageService(
new ObjectMapper(),
new NoopRegistry(),
database.context,
Clock.systemDefaultZone(),
new SqlRetryProperties(),
1,
"default",
new Front50SqlProperties());

// Configure PipelineDAO properties
pipelineDAOConfigProperties.setRefreshMs(0);
pipelineDAOConfigProperties.setShouldWarmCache(false);

DefaultPipelineDAO pipelineDAO =
new DefaultPipelineDAO(
storageService,
scheduler,
new DefaultObjectKeyLoader(storageService),
pipelineDAOConfigProperties,
new NoopRegistry(),
CircuitBreakerRegistry.ofDefaults());

// refreshing to initialize the cache with an empty set
pipelineDAO.all(true);

return pipelineDAO;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -201,9 +201,11 @@ public Collection<Pipeline> getTriggeredPipelines(
@RequestMapping(value = "{application:.+}", method = RequestMethod.GET)
public List<Pipeline> listByApplication(
@PathVariable(value = "application") String application,
@RequestParam(value = "pipelineNameFilter", required = false) String pipelineNameFilter,
@RequestParam(required = false, value = "refresh", defaultValue = "true") boolean refresh) {
List<Pipeline> pipelines =
new ArrayList<>(pipelineDAO.getPipelinesByApplication(application, refresh));
new ArrayList<>(
pipelineDAO.getPipelinesByApplication(application, pipelineNameFilter, refresh));

pipelines.sort(
(p1, p2) -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -383,6 +383,13 @@ class PipelineControllerSpec extends Specification {
.collect(Collectors.toList())
}

@Override
Collection<Pipeline> getPipelinesByApplication(String application, String pipelineFilter, boolean refresh) {
map.values().stream()
.filter({ p -> p.getApplication().equalsIgnoreCase(application) })
.collect(Collectors.toList())
}

@Override
Pipeline getPipelineByName(String application, String pipelineName, boolean refresh) {
map.values().stream()
Expand Down
Loading

0 comments on commit 51f2d15

Please sign in to comment.