Skip to content

Commit

Permalink
Fix: use physical instead of logical name to get available interval (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
garyluoex authored and cdeszaq committed Apr 17, 2017
1 parent b9529e9 commit c6bbe69
Show file tree
Hide file tree
Showing 26 changed files with 581 additions and 398 deletions.
12 changes: 11 additions & 1 deletion CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,11 @@ Current
-------
### Added:

- [Fix to use physical name instead of logical name to retrieve available interval](https://github.com/yahoo/fili/pull/226)
* Added `PhysicalDataSourceConstraint` class to capture physical names of columns for retrieving available intervals

- [BaseCompositePhysicalTable](https://github.com/yahoo/fili/pull/242)
* `ConcretePhysicalTable` provides common operations, such as validating coarsest ZonedTimeGrain, for composite
* `BaseCompositePhysicalTable` provides common operations, such as validating coarsest ZonedTimeGrain, for composite
tables.

- [Add Reciprocal `satisfies()` relationship complementing `satisfiedBy()` on Granularity](https://github.com/yahoo/fili/issues/222)
Expand Down Expand Up @@ -77,6 +80,13 @@ Current

### Changed:

- [Fix to use physical name instead of logical name to retrieve available interval](https://github.com/yahoo/fili/pull/226)
* `getAllAvailbleIntervals` in `ConcreteAvailability` no longer filters out un-configured columns, instead table's `getAllAvailbleIntervals` does
* `getAvailbleIntervals` in `Availbality` now takes `PhysicalDataSourceConstraint` instead of `DataSourceConstraint`
* `Availability` no longer takes a set of columns on the table, only table needs to know
* `getAllAvailbleIntervals` in `Availability` now returns a map of column physical name string to interval list instead of column to interval list
* `TestDataSourceMetadataService` now takes map from string to list of intervals instead of column to list of intervals for constructor

- [Reduced number of queries sent by `LuceneSearchProvider` by 50% in the common case](https://github.com/yahoo/fili/pull/234)
* Before, we were using `IndexSearcher::count` to get the total number of documents, which spawned an entire second query
(so two Lucene queries rather than one when requesting the first page of results). We now pull that information from
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
/**
* An implementation of BasePhysicalTable that contains multiple tables.
*/
public class BaseCompositePhysicalTable extends BasePhysicalTable {
public abstract class BaseCompositePhysicalTable extends BasePhysicalTable {

private static final Logger LOG = LoggerFactory.getLogger(BaseCompositePhysicalTable.class);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
import com.yahoo.bard.webservice.data.time.ZonedTimeGrain;
import com.yahoo.bard.webservice.table.availability.Availability;
import com.yahoo.bard.webservice.table.resolver.DataSourceConstraint;
import com.yahoo.bard.webservice.table.resolver.PhysicalDataSourceConstraint;
import com.yahoo.bard.webservice.util.IntervalUtils;
import com.yahoo.bard.webservice.util.SimplifiedIntervalList;

Expand All @@ -16,6 +17,9 @@

import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;

import javax.validation.constraints.NotNull;

Expand Down Expand Up @@ -79,12 +83,35 @@ public DateTime getTableAlignment() {

@Override
public Map<Column, List<Interval>> getAllAvailableIntervals() {
return getAvailability().getAllAvailableIntervals();
Map<String, List<Interval>> availableIntervals = getAvailability().getAllAvailableIntervals();

return getSchema().getColumns().stream()
.collect(Collectors.toMap(
Function.identity(),
column -> availableIntervals.getOrDefault(
getSchema().getPhysicalColumnName(column.getName()),
new SimplifiedIntervalList()
)
));
}

@Override
public SimplifiedIntervalList getAvailableIntervals(DataSourceConstraint constraint) {
return getAvailability().getAvailableIntervals(constraint);

Set<String> tableColumnNames = getSchema().getColumnNames();

// Validate that the requested columns are answerable by the current table
if (!constraint.getAllColumnNames().stream().allMatch(tableColumnNames::contains)) {
String message = String.format(
"Received invalid request requesting for columns: %s that is not available in this table: %s",
constraint.getAllColumnNames().stream()
.filter(name -> !tableColumnNames.contains(name))
.collect(Collectors.joining(",")), getName());
LOG.error(message);
throw new RuntimeException(message);
}

return getAvailability().getAvailableIntervals(new PhysicalDataSourceConstraint(constraint, getSchema()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@

import java.util.LinkedHashSet;
import java.util.Objects;
import java.util.stream.Collectors;

/**
* A parent class for most schema implementations.
Expand All @@ -33,6 +34,17 @@ public LinkedHashSet<Column> getColumns() {
return columns;
}

/**
* Get the names of the columns returned by getColumns method.
*
* @return linked hash set of column names in this schema
*/
public LinkedHashSet<String> getColumnNames() {
return getColumns().stream()
.map(Column::getName)
.collect(Collectors.toCollection(LinkedHashSet::new));
}

@Override
public Granularity getGranularity() {
return granularity;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ public ConcretePhysicalTable(
timeGrain,
columns,
logicalToPhysicalColumnNames,
new ConcreteAvailability(name, columns, metadataService)
new ConcreteAvailability(name, metadataService)
);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public PermissiveConcretePhysicalTable(
timeGrain,
columns,
logicalToPhysicalColumnNames,
new PermissiveAvailability(name, columns, dataSourceMetadataService)
new PermissiveAvailability(name, dataSourceMetadataService)
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public interface PhysicalTable extends Table {
DateTime getTableAlignment();

/**
* Getter for all the available intervals for the corresponding column.
* Getter for all the available intervals for the corresponding columns configured on the table.
*
* @return map of column to set of available intervals
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
/**
* The schema for a physical table.
*/
public class PhysicalTableSchema extends BaseSchema implements Schema {
public class PhysicalTableSchema extends BaseSchema {

private final ZonedTimeGrain timeGrain;
private final Map<String, String> logicalToPhysicalColumnNames;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,8 +3,7 @@
package com.yahoo.bard.webservice.table.availability;

import com.yahoo.bard.webservice.data.config.names.TableName;
import com.yahoo.bard.webservice.table.Column;
import com.yahoo.bard.webservice.table.resolver.DataSourceConstraint;
import com.yahoo.bard.webservice.table.resolver.PhysicalDataSourceConstraint;
import com.yahoo.bard.webservice.util.SimplifiedIntervalList;

import org.joda.time.Interval;
Expand All @@ -30,14 +29,14 @@ public interface Availability {
*
* @return The intervals, by column, available.
*/
Map<Column, List<Interval>> getAllAvailableIntervals();
Map<String, List<Interval>> getAllAvailableIntervals();

/**
* Fetch a set of intervals given a set of column name in DataSourceConstraint.
*
* @param constraint Data constraint containing columns and api filters
* @param constraint Physical data source constraint containing column's physical name, metrics names, api filters
*
* @return A simplified list of intervals associated with all column in constraint, empty if column is missing
*/
SimplifiedIntervalList getAvailableIntervals(DataSourceConstraint constraint);
SimplifiedIntervalList getAvailableIntervals(PhysicalDataSourceConstraint constraint);
}
Original file line number Diff line number Diff line change
Expand Up @@ -4,22 +4,17 @@

import com.yahoo.bard.webservice.data.config.names.TableName;
import com.yahoo.bard.webservice.metadata.DataSourceMetadataService;
import com.yahoo.bard.webservice.table.Column;
import com.yahoo.bard.webservice.table.resolver.DataSourceConstraint;
import com.yahoo.bard.webservice.table.resolver.PhysicalDataSourceConstraint;
import com.yahoo.bard.webservice.util.IntervalUtils;
import com.yahoo.bard.webservice.util.SimplifiedIntervalList;

import com.google.common.collect.ImmutableSet;

import org.joda.time.Interval;

import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;

import javax.validation.constraints.NotNull;

Expand All @@ -29,29 +24,23 @@
public class ConcreteAvailability implements Availability {

private final TableName name;
private final Set<Column> columns;
private final DataSourceMetadataService metadataService;

private final Set<String> columnNames;
private final Set<TableName> dataSourceNames;

/**
* Constructor.
*
* @param tableName The name of the table and data source associated with this Availability
* @param columns The columns associated with the table and availability
* @param metadataService A service containing the datasource segment data
*/
public ConcreteAvailability(
@NotNull TableName tableName,
@NotNull Set<Column> columns,
@NotNull DataSourceMetadataService metadataService
) {
this.name = tableName;
this.columns = ImmutableSet.copyOf(columns);
this.metadataService = metadataService;

this.columnNames = columns.stream().map(Column::getName).collect(Collectors.toSet());
this.dataSourceNames = Collections.singleton(name);
}

Expand All @@ -61,59 +50,33 @@ public Set<TableName> getDataSourceNames() {
}

@Override
public Map<Column, List<Interval>> getAllAvailableIntervals() {

Map<String, List<Interval>> allAvailableIntervals = getAvailableIntervalsByTable();
return columns.stream()
.collect(
Collectors.toMap(
Function.identity(),
column -> new SimplifiedIntervalList(
allAvailableIntervals.getOrDefault(column.getName(), Collections.emptyList())
)
)
);
public Map<String, List<Interval>> getAllAvailableIntervals() {
return metadataService.getAvailableIntervalsByTable(name);
}

@Override
public SimplifiedIntervalList getAvailableIntervals(DataSourceConstraint constraint) {
public SimplifiedIntervalList getAvailableIntervals(PhysicalDataSourceConstraint constraint) {

Set<String> requestColumns = constraint.getAllColumnNames().stream()
.filter(columnNames::contains)
.collect(Collectors.toSet());
Set<String> requestColumns = constraint.getAllColumnPhysicalNames();

if (requestColumns.isEmpty()) {
return new SimplifiedIntervalList();
}

Map<String, List<Interval>> allAvailableIntervals = getAvailableIntervalsByTable();
Map<String, List<Interval>> allAvailableIntervals = getAllAvailableIntervals();

// Need to ensure requestColumns is not empty in order to prevent returning null by reduce operation
return new SimplifiedIntervalList(
requestColumns.stream()
.map(columnName -> allAvailableIntervals.getOrDefault(columnName, Collections.emptyList()))
.map(physicalName -> allAvailableIntervals.getOrDefault(
physicalName,
new SimplifiedIntervalList()
))
.map(intervals -> (Collection<Interval>) intervals)
.reduce(null, IntervalUtils::getOverlappingSubintervals)
);
}

/**
* Retrieves the most up to date column to available interval map from data source metadata service.
*
* @return map of column name to list of avialable intervals
*/
protected Map<String, List<Interval>> getAvailableIntervalsByTable() {
return metadataService.getAvailableIntervalsByTable(name);
}

/**
* Returns the configured columns in their String representation.
*
* @return the configured columns in their String representation
*/
protected Set<String> getColumnNames() {
return columnNames;
}

/**
* Returns the name of the table and data source associated with this Availability.
Expand All @@ -126,11 +89,8 @@ protected TableName getName() {

@Override
public String toString() {
return String.format("ConcreteAvailability with table name = %s and Configured columns = [%s]",
name.asName(),
columns.stream()
.map(Column::getName)
.collect(Collectors.joining(", "))
return String.format("ConcreteAvailability for table: %s",
name.asName()
);
}
}
Loading

0 comments on commit c6bbe69

Please sign in to comment.