@Override protected StorageKey initialValue() { return new StorageKey(descriptor.getPartitionStrategy()); } };
public StorageKey apply(Path path) { return reusableKey.reuseFor(path, convert); } }
public Path fromKey(StorageKey key) { final StringBuilder pathBuilder = new StringBuilder(); final List<FieldPartitioner> partitioners = Accessor.getDefault().getFieldPartitioners(key.getPartitionStrategy()); for (int i = 0; i < partitioners.size(); i++) { final FieldPartitioner fp = partitioners.get(i); if (i != 0) { pathBuilder.append(Path.SEPARATOR_CHAR); } @SuppressWarnings("unchecked") String dirname = dirnameForValue(fp, key.get(i)); pathBuilder.append(dirname); } return new Path(pathBuilder.toString()); }
@Override @SuppressWarnings("unchecked") public boolean apply(StorageKey key) { if (key == null) { return false; } // this is fail-fast: if the key fails a constraint, then drop it //only check for the number of values in the key in case the key does not //have all fields populated yet for (int i = 0; i < key.size(); i += 1) { Object pValue = key.get(i); if (!partitionPredicates.get(i).apply(pValue)) { return false; } } for (Predicate<Marker> timePredicate : timePredicates) { if (!timePredicate.apply(key)) { return false; } } // if we made it this far, everything passed return true; }
public StorageKey keyFor(E object, @Nullable Map<String, Object> provided, StorageKey reuse) { Preconditions.checkNotNull(reuse, "Cannot use null key"); PartitionStrategy strategy = reuse.getPartitionStrategy(); List<FieldPartitioner> partitioners = Accessor.getDefault().getFieldPartitioners(strategy); for (int i = 0, n = partitioners.size(); i < n; i += 1) { reuse.replace(i, partitionValue(object, provided, partitioners.get(i))); } return reuse; }
accessor = DataModelUtil.accessor( dataset.getType(), dataset.getDescriptor().getSchema()); key = new StorageKey(partitionStrategy); key.reuseFor(message, accessor); int i = 0; for (FieldPartitioner fp : Accessor.getDefault().getFieldPartitioners(partitionStrategy)) { hdrs.put(PARTITION_PREFIX + fp.getName(), PathConversion.valueToString(fp, key.get(i++)));
.build(); StorageKey expected = new StorageKey(strategy); expected.replaceValues((List) Lists.newArrayList(1L, 2013, 11)); new Path(ROOT_PATH, "id=1/year=2013/month=11"), new StorageKey(strategy))); expected.replaceValues((List) Lists.newArrayList(1L, 2013)); new Path(ROOT_PATH, "id=1/year=2013"), new StorageKey(strategy))); expected.replaceValues((List) Lists.newArrayList(1L)); new Path(ROOT_PATH, "id=1"), new StorageKey(strategy))); expected.replaceValues(Collections.emptyList()); ROOT_PATH, new StorageKey(strategy)));
@Test public void testStorageKeyForReflectedEntity() { Station station = new Station("KCASANFR291", 37.776, -122.418); StorageKey key = new StorageKey.Builder(strategy) .add("position.latitude", 0) .add("position.longitude", 0) .build(); StorageKey expected = StorageKey.copy(key); expected.replace(0, 37.776); expected.replace(1, -122.418); key.reuseFor(station, DataModelUtil.accessor(Station.class, schema)); Assert.assertEquals(expected, key); } }
@Test @SuppressWarnings("unchecked") public void toDirNameIdentityWithNonString() { PartitionStrategy strategy = new PartitionStrategy.Builder() .identity("id") .build(); StorageKey expected = new StorageKey(strategy); expected.replace(0, 0L); Assert.assertEquals("Should convert to schema type", expected, convert.toKey(new Path("id=0"), new StorageKey(strategy))); } }
@Override public Object get(int i) { return key.get(i); } }
public StorageKey toKey(Path fromPath, StorageKey storage) { final List<FieldPartitioner> partitioners = Accessor.getDefault().getFieldPartitioners(storage.getPartitionStrategy()); final List<Object> values = Lists.newArrayList( new Object[partitioners.size()]); Path currentPath = fromPath; int index = partitioners.size() - 1; while (currentPath != null && index >= 0) { values.set(index, valueForDirname( (FieldPartitioner<?, ?>) partitioners.get(index), currentPath.getName())); // update currentPath = currentPath.getParent(); index -= 1; } storage.replaceValues(values); return storage; }
@Override public void put(int i, Object v) { key.replace(i, v); }
@Override public void write(E entity) { Preconditions.checkState(state.equals(ReaderWriterState.OPEN), "Attempt to write to a writer in state:%s", state); accessor.keyFor(entity, provided, reusedKey); DatasetWriter<E> writer = cachedWriters.getIfPresent(reusedKey); if (writer == null) { // avoid checking in every whether the entity belongs in the view by only // checking when a new writer is created Preconditions.checkArgument(view.includes(entity), "View %s does not include entity %s", view, entity); // get a new key because it is stored in the cache StorageKey key = StorageKey.copy(reusedKey); try { writer = cachedWriters.getUnchecked(key); } catch (UncheckedExecutionException ex) { throw new IllegalArgumentException( "Problem creating view for entity: " + entity, ex.getCause()); } } writer.write(entity); }
accessor = DataModelUtil.accessor( dataset.getType(), dataset.getDescriptor().getSchema()); key = new StorageKey(partitionStrategy); key.reuseFor(message, accessor); int i = 0; for (FieldPartitioner fp : Accessor.getDefault().getFieldPartitioners(partitionStrategy)) { hdrs.put(PARTITION_PREFIX + fp.getName(), PathConversion.valueToString(fp, key.get(i++)));
@Test @SuppressWarnings("unchecked") public void testToKey() { PartitionStrategy strategy = new PartitionStrategy.Builder() .year("timestamp") .month("timestamp") .day("timestamp") .build(); StorageKey expected = new StorageKey(strategy); expected.replaceValues((List) Lists.newArrayList(2013, 11, 5)); Assert.assertEquals(expected, convert.toKey( new Path("year=2013/month=11/day=5"), new StorageKey(strategy))); }
@Test public void testStorageKeyForGenericEntity() { GenericData.Record position = new GenericData.Record( schema.getField("position").schema()); position.put("latitude", 37.776); position.put("longitude", -122.418); GenericData.Record generic = new GenericData.Record(schema); generic.put("name", "KCASANFR291"); generic.put("position", position); StorageKey key = new StorageKey.Builder(strategy) .add("position.latitude", 0) .add("position.longitude", 0) .build(); StorageKey expected = StorageKey.copy(key); expected.replace(0, 37.776); expected.replace(1, -122.418); key.reuseFor(generic, DataModelUtil.accessor(GenericRecord.class, schema)); Assert.assertEquals(expected, key); }
@Override @SuppressWarnings("unchecked") public int compareTo(StorageKey other) { if (other == null) { throw new NullPointerException("Cannot compare a StorageKey with null"); } else if (!strategy.equals(other.strategy)) { throw new RuntimeException("PartitionStrategy does not match"); } final List<FieldPartitioner> partitioners = Accessor.getDefault().getFieldPartitioners(strategy); for (int i = 0; i < partitioners.size(); i += 1) { final FieldPartitioner fp = partitioners.get(i); final int cmp = fp.compare(get(i), other.get(i)); if (cmp != 0) { return cmp; } } return 0; }
public StorageKey toKey(Path fromPath, StorageKey storage) { final List<FieldPartitioner> partitioners = Accessor.getDefault().getFieldPartitioners(storage.getPartitionStrategy()); storage.replaceValues(values); return storage;
/** * A convenience method to make a copy of a {@link StorageKey}. * * This is not a deep copy. * * @param toCopy a {@code StorageKey} to copy * @return a new StorageKey with the same {@link PartitionStrategy} and content */ public static StorageKey copy(StorageKey toCopy) { return new StorageKey(toCopy.strategy, Lists.newArrayList(toCopy.values)); }
@Test @SuppressWarnings("unchecked") public void toDirNameIdentityWithSlashes() { PartitionStrategy strategy = new PartitionStrategy.Builder() .identity("name") .identity("address") .build(); StorageKey key = new StorageKey(strategy); key.replaceValues((List) Lists.newArrayList("John Doe", "NY/USA")); Assert.assertEquals( new Path("name_copy=John+Doe/address_copy=NY%2FUSA"), convert.fromKey(key)); }