import static com.google.common.base.Preconditions.checkState;
import com.google.common.base.Optional;
-import io.fd.honeycomb.lisp.translate.read.dump.check.LocatorDumpCheck;
import io.fd.honeycomb.lisp.translate.read.dump.executor.LocatorDumpExecutor;
import io.fd.honeycomb.lisp.translate.read.dump.executor.params.LocatorDumpParams;
import io.fd.honeycomb.lisp.translate.read.dump.executor.params.LocatorDumpParams.LocatorDumpParamsBuilder;
import io.fd.honeycomb.translate.read.ReadFailedException;
import io.fd.honeycomb.translate.spi.read.ListReaderCustomizer;
import io.fd.honeycomb.translate.util.RWUtils;
-import io.fd.honeycomb.translate.vpp.util.FutureJVppCustomizer;
-import io.fd.honeycomb.translate.vpp.util.NamingContext;
import io.fd.honeycomb.translate.util.read.cache.DumpCacheManager;
import io.fd.honeycomb.translate.util.read.cache.exceptions.execution.DumpExecutionFailedException;
+import io.fd.honeycomb.translate.vpp.util.FutureJVppCustomizer;
+import io.fd.honeycomb.translate.vpp.util.NamingContext;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
this.dumpCacheManager =
new DumpCacheManager.DumpCacheManagerBuilder<LispLocatorDetailsReplyDump, LocatorDumpParams>()
.withExecutor(new LocatorDumpExecutor(futureJvpp))
- .withNonEmptyPredicate(new LocatorDumpCheck())
.build();
}
throw new ReadFailedException(id, e);
}
- if (reply.isPresent()) {
- final LispLocatorDetails details = reply.get()
- .lispLocatorDetails
- .stream()
- .filter(a -> a.swIfIndex == referencedInterfaceIndex)
- .collect(RWUtils.singleItemCollector());
+ if (!reply.isPresent() || reply.get().lispLocatorDetails.isEmpty()) {
+ return;
+ }
- final String interfaceRef = interfaceContext.getName(details.swIfIndex, ctx.getMappingContext());
+ final LispLocatorDetails details = reply.get()
+ .lispLocatorDetails
+ .stream()
+ .filter(a -> a.swIfIndex == referencedInterfaceIndex)
+ .collect(RWUtils.singleItemCollector());
- builder.setPriority(Byte.valueOf(details.priority).shortValue());
- builder.setWeight(Byte.valueOf(details.weight).shortValue());
- builder.setInterfaceRef(interfaceRef);
- builder.setKey(new InterfaceKey(interfaceRef));
- }
+ final String interfaceRef = interfaceContext.getName(details.swIfIndex, ctx.getMappingContext());
+
+ builder.setPriority(Byte.valueOf(details.priority).shortValue());
+ builder.setWeight(Byte.valueOf(details.weight).shortValue());
+ builder.setInterfaceRef(interfaceRef);
+ builder.setKey(new InterfaceKey(interfaceRef));
}
@Override
throw new ReadFailedException(id, e);
}
- if (reply.isPresent()) {
- return reply.get()
- .lispLocatorDetails
- .stream()
- .map(a -> new InterfaceKey(interfaceContext.getName(a.swIfIndex, context.getMappingContext())))
- .collect(Collectors.toList());
- } else {
+ if (!reply.isPresent() || reply.get().lispLocatorDetails.isEmpty()) {
return Collections.emptyList();
}
+
+ return reply.get()
+ .lispLocatorDetails
+ .stream()
+ .map(a -> new InterfaceKey(interfaceContext.getName(a.swIfIndex, context.getMappingContext())))
+ .collect(Collectors.toList());
}
@Override
import com.google.common.base.Optional;
import io.fd.honeycomb.lisp.context.util.EidMappingContext;
-import io.fd.honeycomb.lisp.translate.read.dump.check.MappingsDumpCheck;
import io.fd.honeycomb.lisp.translate.read.dump.executor.MappingsDumpExecutor;
import io.fd.honeycomb.lisp.translate.read.dump.executor.params.MappingsDumpParams;
import io.fd.honeycomb.lisp.translate.util.EidTranslator;
this.dumpManager =
new DumpCacheManager.DumpCacheManagerBuilder<LispEidTableDetailsReplyDump, MappingsDumpParams>()
.withExecutor(dumpExecutor)
- .withNonEmptyPredicate(new MappingsDumpCheck())
.build();
}
throw new ReadFailedException(id, e);
}
- if (replyOptional.isPresent()) {
- LOG.debug("Valid dump loaded");
-
- LispEidTableDetails details = replyOptional.get().lispEidTableDetails.stream()
- .filter(a -> compareAddresses(eid.getAddress(),
- getArrayAsEidLocal(valueOf(a.eidType), a.eid).getAddress()))
- .collect(
- RWUtils.singleItemCollector());
-
- //in case of local mappings,locator_set_index stands for interface index
- checkState(locatorSetContext.containsName(details.locatorSetIndex, ctx.getMappingContext()),
- "No Locator Set name found for index %s", details.locatorSetIndex);
- builder.setLocatorSet(locatorSetContext.getName(details.locatorSetIndex, ctx.getMappingContext()));
- builder.setKey(new LocalMappingKey(new MappingId(id.firstKeyOf(LocalMapping.class).getId())));
- builder.setEid(getArrayAsEidLocal(valueOf(details.eidType), details.eid));
- } else {
- LOG.debug("No data dumped");
+ if (!replyOptional.isPresent() || replyOptional.get().lispEidTableDetails.isEmpty()) {
+ return;
}
+
+ LispEidTableDetails details = replyOptional.get().lispEidTableDetails.stream()
+ .filter(a -> compareAddresses(eid.getAddress(),
+ getArrayAsEidLocal(valueOf(a.eidType), a.eid).getAddress()))
+ .collect(
+ RWUtils.singleItemCollector());
+
+ //in case of local mappings,locator_set_index stands for interface index
+ checkState(locatorSetContext.containsName(details.locatorSetIndex, ctx.getMappingContext()),
+ "No Locator Set name found for index %s", details.locatorSetIndex);
+ builder.setLocatorSet(locatorSetContext.getName(details.locatorSetIndex, ctx.getMappingContext()));
+ builder.setKey(new LocalMappingKey(new MappingId(id.firstKeyOf(LocalMapping.class).getId())));
+ builder.setEid(getArrayAsEidLocal(valueOf(details.eidType), details.eid));
}
@Override
throw new ReadFailedException(id, e);
}
- if (replyOptional.isPresent()) {
- LOG.debug("Valid dump loaded");
- return replyOptional.get().lispEidTableDetails.stream()
- //filtering with vni to skip help local mappings that are created in vpp to handle remote mappings(vpp feature)
- .filter(a -> a.vni == vni)
- .map(a -> new LocalMappingKey(
- new MappingId(
- localMappingContext.getId(
- getArrayAsEidLocal(valueOf(a.eidType), a.eid),
- context.getMappingContext()))))
- .collect(Collectors.toList());
- } else {
- LOG.debug("No data dumped");
+ if (!replyOptional.isPresent() || replyOptional.get().lispEidTableDetails.isEmpty()) {
return Collections.emptyList();
}
+
+ return replyOptional.get().lispEidTableDetails.stream()
+ //filtering with vni to skip help local mappings that are created in vpp to handle remote mappings(vpp feature)
+ .filter(a -> a.vni == vni)
+ .map(a -> new LocalMappingKey(
+ new MappingId(
+ localMappingContext.getId(
+ getArrayAsEidLocal(valueOf(a.eidType), a.eid),
+ context.getMappingContext()))))
+ .collect(Collectors.toList());
}
@Override
import static io.fd.honeycomb.translate.util.read.cache.EntityDumpExecutor.NO_PARAMS;
import com.google.common.base.Optional;
-import io.fd.honeycomb.lisp.translate.read.dump.check.LocatorSetsDumpCheck;
import io.fd.honeycomb.lisp.translate.read.dump.executor.LocatorSetsDumpExecutor;
import io.fd.honeycomb.translate.read.ReadContext;
import io.fd.honeycomb.translate.read.ReadFailedException;
this.locatorSetContext = checkNotNull(locatorSetContext, "Locator Set mapping context cannot be null");
this.dumpManager = new DumpCacheManager.DumpCacheManagerBuilder<LispLocatorSetDetailsReplyDump, Void>()
.withExecutor(new LocatorSetsDumpExecutor(futureJvpp))
- .withNonEmptyPredicate(new LocatorSetsDumpCheck())
.build();
}
} catch (DumpExecutionFailedException e) {
throw new ReadFailedException(id, e);
}
- if (!dumpOptional.isPresent()) {
- LOG.warn("No dump present for Locator Set {}", id);
+ if (!dumpOptional.isPresent() || dumpOptional.get().lispLocatorSetDetails.isEmpty()) {
return;
}
return Collections.emptyList();
}
- if (dumpOptional.isPresent()) {
- return dumpOptional.get().lispLocatorSetDetails.stream()
- .map(set -> {
-
- final String locatorSetName = toString(set.lsName);
- //creates mapping for existing locator-set(if it is'nt already existing one)
- if (!locatorSetContext.containsIndex(locatorSetName, context.getMappingContext())) {
- locatorSetContext.addName(set.lsIndex, locatorSetName, context.getMappingContext());
- }
-
- LOG.trace("Locator Set with name: {}, VPP name: {} and index: {} found in VPP",
- locatorSetContext.getName(set.lsIndex, context.getMappingContext()),
- locatorSetName,
- set.lsIndex);
-
- return set;
- })
- .map(set -> new LocatorSetKey(toString(set.lsName)))
- .collect(Collectors.toList());
- } else {
- LOG.warn("No data dumped for Locator Set {}", id);
+ if (!dumpOptional.isPresent() || dumpOptional.get().lispLocatorSetDetails.isEmpty()) {
return Collections.emptyList();
}
+
+ return dumpOptional.get().lispLocatorSetDetails.stream()
+ .map(set -> {
+
+ final String locatorSetName = toString(set.lsName);
+ //creates mapping for existing locator-set(if it is'nt already existing one)
+ if (!locatorSetContext.containsIndex(locatorSetName, context.getMappingContext())) {
+ locatorSetContext.addName(set.lsIndex, locatorSetName, context.getMappingContext());
+ }
+
+ LOG.trace("Locator Set with name: {}, VPP name: {} and index: {} found in VPP",
+ locatorSetContext.getName(set.lsIndex, context.getMappingContext()),
+ locatorSetName,
+ set.lsIndex);
+
+ return set;
+ })
+ .map(set -> new LocatorSetKey(toString(set.lsName)))
+ .collect(Collectors.toList());
}
@Override
import static io.fd.honeycomb.translate.util.read.cache.EntityDumpExecutor.NO_PARAMS;
import com.google.common.base.Optional;
-import io.fd.honeycomb.lisp.translate.read.dump.check.MapResolverDumpCheck;
import io.fd.honeycomb.lisp.translate.read.dump.executor.MapResolversDumpExecutor;
import io.fd.honeycomb.translate.read.ReadContext;
import io.fd.honeycomb.translate.read.ReadFailedException;
import io.fd.honeycomb.translate.spi.read.ListReaderCustomizer;
+import io.fd.honeycomb.translate.util.RWUtils;
import io.fd.honeycomb.translate.util.read.cache.DumpCacheManager;
import io.fd.honeycomb.translate.util.read.cache.exceptions.execution.DumpExecutionFailedException;
import io.fd.honeycomb.translate.vpp.util.AddressTranslator;
super(futureJvpp);
this.dumpManager = new DumpCacheManager.DumpCacheManagerBuilder<LispMapResolverDetailsReplyDump, Void>()
.withExecutor(new MapResolversDumpExecutor((futureJvpp)))
- .withNonEmptyPredicate(new MapResolverDumpCheck())
.build();
}
throw new ReadFailedException(id, e);
}
- if (!dumpOptional.isPresent()) {
+ if (!dumpOptional.isPresent() || dumpOptional.get().lispMapResolverDetails.isEmpty()) {
LOG.warn("No data dumped");
return;
}
final MapResolverKey key = id.firstKeyOf(MapResolver.class);
//revert searched key to match vpp's reversed order ip's
final IpAddress address = reverseAddress(key.getIpAddress());
- final LispMapResolverDetailsReplyDump dump = dumpOptional.get();
-
- //cannot use RWUtils.singleItemCollector(),there is some problem with generic params binding
- java.util.Optional<LispMapResolverDetails> mapResolverOptional =
- dump.lispMapResolverDetails.stream()
+ final LispMapResolverDetails mapResolverDetails =
+ dumpOptional.get().lispMapResolverDetails.stream()
.filter(a -> address
.equals(arrayToIpAddress(byteToBoolean(a.isIpv6), a.ipAddress)))
- .findFirst();
-
- if (mapResolverOptional.isPresent()) {
- LispMapResolverDetails details = mapResolverOptional.get();
-
- builder.setKey(key);
- builder.setIpAddress(
- arrayToIpAddress(byteToBoolean(details.isIpv6), details.ipAddress));
- } else {
- LOG.warn("No data found with matching key");
- }
+ .collect(RWUtils.singleItemCollector());
+ builder.setKey(key);
+ builder.setIpAddress(
+ arrayToIpAddress(byteToBoolean(mapResolverDetails.isIpv6), mapResolverDetails.ipAddress));
}
@Override
throws ReadFailedException {
LOG.debug("Dumping MapResolver...");
- Optional<LispMapResolverDetailsReplyDump> dumpOptional = null;
+ Optional<LispMapResolverDetailsReplyDump> dumpOptional;
try {
dumpOptional = dumpManager.getDump(MAP_RESOLVERS_CACHE_ID, context.getModificationCache(), NO_PARAMS);
} catch (DumpExecutionFailedException e) {
throw new ReadFailedException(id, e);
}
- if (!dumpOptional.isPresent()) {
- LOG.warn("No data dumped");
+ if (!dumpOptional.isPresent() || dumpOptional.get().lispMapResolverDetails.isEmpty()) {
return Collections.emptyList();
}
import com.google.common.base.Optional;
import io.fd.honeycomb.lisp.context.util.EidMappingContext;
-import io.fd.honeycomb.lisp.translate.read.dump.check.MappingsDumpCheck;
import io.fd.honeycomb.lisp.translate.read.dump.executor.MappingsDumpExecutor;
import io.fd.honeycomb.lisp.translate.read.dump.executor.params.MappingsDumpParams;
import io.fd.honeycomb.lisp.translate.read.dump.executor.params.MappingsDumpParams.QuantityType;
this.dumpManager =
new DumpCacheManager.DumpCacheManagerBuilder<LispEidTableDetailsReplyDump, MappingsDumpParams>()
.withExecutor(new MappingsDumpExecutor(futureJvpp))
- .withNonEmptyPredicate(new MappingsDumpCheck())
.build();
}
throw new ReadFailedException(id, e);
}
- if (replyOptional.isPresent()) {
- LOG.debug("Valid dump loaded");
+ if (!replyOptional.isPresent() || replyOptional.get().lispEidTableDetails.isEmpty()) {
+ return;
+ }
- LispEidTableDetails details = replyOptional.get().lispEidTableDetails.stream()
- .filter(a -> compareAddresses(eid.getAddress(),
- getArrayAsEidLocal(valueOf(a.eidType), a.eid).getAddress()))
- .collect(
- RWUtils.singleItemCollector());
+ LOG.debug("Valid dump loaded");
- builder.setEid(getArrayAsEidRemote(valueOf(details.eidType), details.eid));
- builder.setKey(new RemoteMappingKey(new MappingId(id.firstKeyOf(RemoteMapping.class).getId())));
- builder.setTtl(resolveTtl(details.ttl));
- builder.setAuthoritative(
- new RemoteMapping.Authoritative(byteToBoolean(details.authoritative)));
+ LispEidTableDetails details = replyOptional.get().lispEidTableDetails.stream()
+ .filter(a -> compareAddresses(eid.getAddress(),
+ getArrayAsEidLocal(valueOf(a.eidType), a.eid).getAddress()))
+ .collect(
+ RWUtils.singleItemCollector());
- } else {
- LOG.debug("No data dumped");
- }
+ builder.setEid(getArrayAsEidRemote(valueOf(details.eidType), details.eid));
+ builder.setKey(new RemoteMappingKey(new MappingId(id.firstKeyOf(RemoteMapping.class).getId())));
+ builder.setTtl(resolveTtl(details.ttl));
+ builder.setAuthoritative(
+ new RemoteMapping.Authoritative(byteToBoolean(details.authoritative)));
}
//compensate ~0 as default value of ttl
throw new ReadFailedException(id, e);
}
- if (replyOptional.isPresent()) {
- LOG.debug("Valid dump loaded");
- return replyOptional.get()
- .lispEidTableDetails
- .stream()
- .filter(a -> a.vni == vni)
- .map(detail -> new RemoteMappingKey(
- new MappingId(
- remoteMappingContext.getId(
- getArrayAsEidRemote(
- valueOf(detail.eidType), detail.eid),
- context.getMappingContext()))))
- .collect(Collectors.toList());
- } else {
- LOG.debug("No data dumped");
+ if (!replyOptional.isPresent() || replyOptional.get().lispEidTableDetails.isEmpty()) {
return Collections.emptyList();
}
+
+ return replyOptional.get()
+ .lispEidTableDetails
+ .stream()
+ .filter(a -> a.vni == vni)
+ .map(detail -> new RemoteMappingKey(
+ new MappingId(
+ remoteMappingContext.getId(
+ getArrayAsEidRemote(
+ valueOf(detail.eidType), detail.eid),
+ context.getMappingContext()))))
+ .collect(Collectors.toList());
}
@Override
import static io.fd.honeycomb.translate.util.read.cache.EntityDumpExecutor.NO_PARAMS;
import com.google.common.base.Optional;
-import io.fd.honeycomb.lisp.translate.read.dump.check.VniTableDumpCheck;
import io.fd.honeycomb.lisp.translate.read.dump.executor.VniTableDumpExecutor;
import io.fd.honeycomb.translate.read.ReadContext;
import io.fd.honeycomb.translate.read.ReadFailedException;
import io.fd.honeycomb.translate.spi.read.ListReaderCustomizer;
import io.fd.honeycomb.translate.util.RWUtils;
-import io.fd.honeycomb.translate.vpp.util.FutureJVppCustomizer;
import io.fd.honeycomb.translate.util.read.cache.DumpCacheManager;
import io.fd.honeycomb.translate.util.read.cache.exceptions.execution.DumpExecutionFailedException;
+import io.fd.honeycomb.translate.vpp.util.FutureJVppCustomizer;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
super(futureJvpp);
this.dumpManager = new DumpCacheManager.DumpCacheManagerBuilder<LispEidTableMapDetailsReplyDump, Void>()
.withExecutor(new VniTableDumpExecutor(futureJvpp))
- .withNonEmptyPredicate(new VniTableDumpCheck())
.build();
}
throw new ReadFailedException(id, e);
}
- if (!optionalReply.isPresent()) {
+ if (!optionalReply.isPresent() || optionalReply.get().lispEidTableMapDetails.isEmpty()) {
return Collections.emptyList();
}
- LispEidTableMapDetailsReplyDump reply = optionalReply.get();
- LOG.debug("Dumped ...");
-
- // Just transform received details into a list of keys
- final List<VniTableKey> collect = reply.lispEidTableMapDetails.stream().map(VniTableCustomizer::detailsToKey)
+ return optionalReply.get().lispEidTableMapDetails.stream().map(VniTableCustomizer::detailsToKey)
.collect(Collectors.toList());
- LOG.debug("All IDs found: {} ...", collect);
- return collect;
}
@Override
throw new ReadFailedException(id, e);
}
- if (!optionalReply.isPresent()) {
+ if (!optionalReply.isPresent() || optionalReply.get().lispEidTableMapDetails.isEmpty()) {
return;
}
//transforming right away to single detail(specific request should do the magic)
- LispEidTableMapDetails details =
+ final LispEidTableMapDetails details =
optionalReply.get().lispEidTableMapDetails.stream().filter(a -> detailsToKey(a).equals(key))
.collect(RWUtils.singleItemCollector());
+++ /dev/null
-/*
- * Copyright (c) 2016 Cisco and/or its affiliates.
- *
- * 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 io.fd.honeycomb.lisp.translate.read.dump.check;
-
-import io.fd.honeycomb.translate.util.read.cache.EntityDumpNonEmptyCheck;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.DumpCheckFailedException;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.i.DumpEmptyException;
-import org.openvpp.jvpp.core.dto.LispLocatorDetailsReplyDump;
-
-public class LocatorDumpCheck implements EntityDumpNonEmptyCheck<LispLocatorDetailsReplyDump> {
-
- @Override
- public void assertNotEmpty(final LispLocatorDetailsReplyDump data) throws DumpCheckFailedException {
-
- if (data == null) {
- throw new DumpEmptyException("Locator dump is null");
- }
-
- if (data.lispLocatorDetails == null) {
- throw new DumpEmptyException("Locator dump is empty");
- }
- }
-}
+++ /dev/null
-/*
- * Copyright (c) 2016 Cisco and/or its affiliates.
- *
- * 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 io.fd.honeycomb.lisp.translate.read.dump.check;
-
-
-import io.fd.honeycomb.translate.util.read.cache.EntityDumpNonEmptyCheck;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.DumpCheckFailedException;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.i.DumpEmptyException;
-import org.openvpp.jvpp.core.dto.LispLocatorSetDetailsReplyDump;
-
-public class LocatorSetsDumpCheck implements EntityDumpNonEmptyCheck<LispLocatorSetDetailsReplyDump> {
-
- @Override
- public void assertNotEmpty(final LispLocatorSetDetailsReplyDump data) throws DumpCheckFailedException {
-
- if (data == null) {
- throw new DumpEmptyException("Locator sets dump is null");
- }
-
- if (data.lispLocatorSetDetails == null) {
- throw new DumpEmptyException("Locator sets dump is empty");
- }
- }
-}
+++ /dev/null
-/*
- * Copyright (c) 2016 Cisco and/or its affiliates.
- *
- * 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 io.fd.honeycomb.lisp.translate.read.dump.check;
-
-
-import io.fd.honeycomb.translate.util.read.cache.EntityDumpNonEmptyCheck;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.DumpCheckFailedException;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.i.DumpEmptyException;
-import org.openvpp.jvpp.core.dto.LispMapResolverDetailsReplyDump;
-
-public class MapResolverDumpCheck implements EntityDumpNonEmptyCheck<LispMapResolverDetailsReplyDump> {
-
- @Override
- public void assertNotEmpty(final LispMapResolverDetailsReplyDump data) throws DumpCheckFailedException {
- if (data == null) {
- throw new DumpEmptyException("Map resolvers dump is null");
- }
-
- if (data.lispMapResolverDetails == null) {
- throw new DumpEmptyException("Map resolvers dump is empty");
- }
- }
-}
+++ /dev/null
-/*
- * Copyright (c) 2016 Cisco and/or its affiliates.
- *
- * 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 io.fd.honeycomb.lisp.translate.read.dump.check;
-
-
-import io.fd.honeycomb.translate.util.read.cache.EntityDumpNonEmptyCheck;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.DumpCheckFailedException;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.i.DumpEmptyException;
-import org.openvpp.jvpp.core.dto.LispEidTableDetailsReplyDump;
-
-/**
- * Common dump check for both local and remote mappings
- */
-public class MappingsDumpCheck implements EntityDumpNonEmptyCheck<LispEidTableDetailsReplyDump> {
-
- @Override
- public void assertNotEmpty(final LispEidTableDetailsReplyDump data) throws DumpCheckFailedException {
- if (data == null) {
- throw new DumpEmptyException("Mappings dump is null");
- }
-
- if (data.lispEidTableDetails == null) {
- throw new DumpEmptyException("Mappings dump is empty");
- }
- }
-}
+++ /dev/null
-/*
- * Copyright (c) 2016 Cisco and/or its affiliates.
- *
- * 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 io.fd.honeycomb.lisp.translate.read.dump.check;
-
-
-import io.fd.honeycomb.translate.util.read.cache.EntityDumpNonEmptyCheck;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.DumpCheckFailedException;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.i.DumpEmptyException;
-import org.openvpp.jvpp.core.dto.LispEidTableMapDetailsReplyDump;
-
-public class VniTableDumpCheck implements EntityDumpNonEmptyCheck<LispEidTableMapDetailsReplyDump> {
-
- @Override
- public void assertNotEmpty(final LispEidTableMapDetailsReplyDump data) throws DumpCheckFailedException {
- if (data == null) {
- throw new DumpEmptyException("Vni table dump is null");
- }
-
- if (data.lispEidTableMapDetails == null) {
- throw new DumpEmptyException("Bni table dump is empty");
- }
- }
-}
import static java.nio.charset.StandardCharsets.UTF_8;
import com.google.common.base.Optional;
-import io.fd.honeycomb.lisp.translate.read.dump.check.LocatorSetsDumpCheck;
import io.fd.honeycomb.lisp.translate.read.dump.executor.LocatorSetsDumpExecutor;
import io.fd.honeycomb.translate.ModificationCache;
import io.fd.honeycomb.translate.spi.write.ListWriterCustomizer;
this.locatorSetContext = checkNotNull(locatorSetContext, "Locator set context cannot be null");
this.dumpManager = new DumpCacheManager.DumpCacheManagerBuilder<LispLocatorSetDetailsReplyDump, Void>()
.withExecutor(new LocatorSetsDumpExecutor(futureJvpp))
- .withNonEmptyPredicate(new LocatorSetsDumpCheck())
.build();
}
+++ /dev/null
-package io.fd.honeycomb.lisp.translate.read.dump.check;
-
-import io.fd.honeycomb.vpp.test.read.DumpCheckTest;
-import org.openvpp.jvpp.core.dto.LispLocatorDetailsReplyDump;
-
-public class LocatorDumpCheckTest extends DumpCheckTest<LocatorDumpCheck, LispLocatorDetailsReplyDump> {
-
- @Override
- protected LocatorDumpCheck initCheck() {
- return new LocatorDumpCheck();
- }
-
- @Override
- protected LispLocatorDetailsReplyDump initEmptyData() {
- final LispLocatorDetailsReplyDump replyDump = new LispLocatorDetailsReplyDump();
- replyDump.lispLocatorDetails = null;
-
- return replyDump;
- }
-
- @Override
- protected LispLocatorDetailsReplyDump initValidData() {
- return new LispLocatorDetailsReplyDump();
- }
-}
\ No newline at end of file
+++ /dev/null
-package io.fd.honeycomb.lisp.translate.read.dump.check;
-
-import io.fd.honeycomb.vpp.test.read.DumpCheckTest;
-import org.openvpp.jvpp.core.dto.LispLocatorSetDetailsReplyDump;
-
-
-public class LocatorSetsDumpCheckTest extends DumpCheckTest<LocatorSetsDumpCheck, LispLocatorSetDetailsReplyDump> {
-
- @Override
- protected LocatorSetsDumpCheck initCheck() {
- return new LocatorSetsDumpCheck();
- }
-
- @Override
- protected LispLocatorSetDetailsReplyDump initEmptyData() {
- final LispLocatorSetDetailsReplyDump replyDump = new LispLocatorSetDetailsReplyDump();
- replyDump.lispLocatorSetDetails = null;
-
- return replyDump;
- }
-
- @Override
- protected LispLocatorSetDetailsReplyDump initValidData() {
- return new LispLocatorSetDetailsReplyDump();
- }
-}
\ No newline at end of file
+++ /dev/null
-package io.fd.honeycomb.lisp.translate.read.dump.check;
-
-import io.fd.honeycomb.vpp.test.read.DumpCheckTest;
-import org.openvpp.jvpp.core.dto.LispMapResolverDetailsReplyDump;
-
-
-public class MapResolverDumpCheckTest extends DumpCheckTest<MapResolverDumpCheck, LispMapResolverDetailsReplyDump> {
-
- @Override
- protected MapResolverDumpCheck initCheck() {
- return new MapResolverDumpCheck();
- }
-
- @Override
- protected LispMapResolverDetailsReplyDump initEmptyData() {
- final LispMapResolverDetailsReplyDump replyDump = new LispMapResolverDetailsReplyDump();
- replyDump.lispMapResolverDetails = null;
-
- return replyDump;
- }
-
- @Override
- protected LispMapResolverDetailsReplyDump initValidData() {
- return new LispMapResolverDetailsReplyDump();
- }
-}
\ No newline at end of file
+++ /dev/null
-package io.fd.honeycomb.lisp.translate.read.dump.check;
-
-import io.fd.honeycomb.vpp.test.read.DumpCheckTest;
-import org.openvpp.jvpp.core.dto.LispEidTableDetailsReplyDump;
-
-
-public class MappingsDumpCheckTest extends DumpCheckTest<MappingsDumpCheck, LispEidTableDetailsReplyDump> {
-
- @Override
- protected MappingsDumpCheck initCheck() {
- return new MappingsDumpCheck();
- }
-
- @Override
- protected LispEidTableDetailsReplyDump initEmptyData() {
- final LispEidTableDetailsReplyDump replyDump = new LispEidTableDetailsReplyDump();
- replyDump.lispEidTableDetails = null;
-
- return replyDump;
- }
-
- @Override
- protected LispEidTableDetailsReplyDump initValidData() {
- return new LispEidTableDetailsReplyDump();
- }
-}
\ No newline at end of file
+++ /dev/null
-package io.fd.honeycomb.lisp.translate.read.dump.check;
-
-import io.fd.honeycomb.vpp.test.read.DumpCheckTest;
-import org.openvpp.jvpp.core.dto.LispEidTableMapDetailsReplyDump;
-
-
-public class VniTableDumpCheckTest extends DumpCheckTest<VniTableDumpCheck, LispEidTableMapDetailsReplyDump> {
-
- @Override
- protected VniTableDumpCheck initCheck() {
- return new VniTableDumpCheck();
- }
-
- @Override
- protected LispEidTableMapDetailsReplyDump initEmptyData() {
- final LispEidTableMapDetailsReplyDump replyDump = new LispEidTableMapDetailsReplyDump();
- replyDump.lispEidTableMapDetails = null;
-
- return replyDump;
- }
-
- @Override
- protected LispEidTableMapDetailsReplyDump initValidData() {
- return new LispEidTableMapDetailsReplyDump();
- }
-}
\ No newline at end of file
import io.fd.honeycomb.translate.util.read.cache.DumpCacheManager;
import io.fd.honeycomb.translate.util.read.cache.exceptions.execution.DumpExecutionFailedException;
import io.fd.honeycomb.translate.v3po.interfacesstate.ip.dump.AddressDumpExecutor;
-import io.fd.honeycomb.translate.v3po.interfacesstate.ip.dump.check.AddressDumpCheck;
import io.fd.honeycomb.translate.v3po.interfacesstate.ip.dump.params.AddressDumpParams;
import io.fd.honeycomb.translate.vpp.util.FutureJVppCustomizer;
import io.fd.honeycomb.translate.vpp.util.NamingContext;
this.interfaceContext = checkNotNull(interfaceContext, "interfaceContext should not be null");
this.dumpManager = new DumpCacheManager.DumpCacheManagerBuilder<IpAddressDetailsReplyDump, AddressDumpParams>()
.withExecutor(new AddressDumpExecutor(futureJVppCore))
- .withNonEmptyPredicate(new AddressDumpCheck())
.build();
}
throw new ReadFailedException(id, e);
}
- if (dumpOptional.isPresent()) {
- final Optional<IpAddressDetails> ipAddressDetails =
- findIpAddressDetailsByIp(dumpOptional, id.firstKeyOf(Address.class).getIp());
-
- if (ipAddressDetails.isPresent()) {
- final IpAddressDetails detail = ipAddressDetails.get();
- builder.setIp(arrayToIpv4AddressNoZone(detail.ip))
- .setSubnet(
- new PrefixLengthBuilder().setPrefixLength(Short.valueOf(detail.prefixLength)).build());
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Attributes for {} interface (id={}) address {} successfully read: {}",
- interfaceName, interfaceIndex, id, builder.build());
- }
+ if (!dumpOptional.isPresent() || dumpOptional.get().ipAddressDetails.isEmpty()) {
+ return;
+ }
+ final Optional<IpAddressDetails> ipAddressDetails =
+ findIpAddressDetailsByIp(dumpOptional, id.firstKeyOf(Address.class).getIp());
+
+ if (ipAddressDetails.isPresent()) {
+ final IpAddressDetails detail = ipAddressDetails.get();
+ builder.setIp(arrayToIpv4AddressNoZone(detail.ip))
+ .setSubnet(
+ new PrefixLengthBuilder().setPrefixLength(Short.valueOf(detail.prefixLength)).build());
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Attributes for {} interface (id={}) address {} successfully read: {}",
+ interfaceName, interfaceIndex, id, builder.build());
}
}
-
-
}
@Override
import io.fd.honeycomb.translate.util.read.cache.DumpCacheManager;
import io.fd.honeycomb.translate.util.read.cache.exceptions.execution.DumpExecutionFailedException;
import io.fd.honeycomb.translate.v3po.interfacesstate.ip.dump.AddressDumpExecutor;
-import io.fd.honeycomb.translate.v3po.interfacesstate.ip.dump.check.AddressDumpCheck;
import io.fd.honeycomb.translate.v3po.interfacesstate.ip.dump.params.AddressDumpParams;
import io.fd.honeycomb.translate.vpp.util.FutureJVppCustomizer;
import io.fd.honeycomb.translate.vpp.util.NamingContext;
this.interfaceContext = checkNotNull(interfaceContext, "interfaceContext should not be null");
this.dumpManager = new DumpCacheManager.DumpCacheManagerBuilder<IpAddressDetailsReplyDump, AddressDumpParams>()
.withExecutor(new AddressDumpExecutor(futureJVppCore))
- .withNonEmptyPredicate(new AddressDumpCheck())
.build();
}
+++ /dev/null
-package io.fd.honeycomb.translate.v3po.interfacesstate.ip.dump.check;
-
-import io.fd.honeycomb.translate.util.read.cache.EntityDumpNonEmptyCheck;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.DumpCheckFailedException;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.i.DumpEmptyException;
-import org.openvpp.jvpp.core.dto.IpAddressDetailsReplyDump;
-
-public class AddressDumpCheck implements EntityDumpNonEmptyCheck<IpAddressDetailsReplyDump> {
-
- @Override
- public void assertNotEmpty(final IpAddressDetailsReplyDump data) throws DumpCheckFailedException {
- if (data == null || data.ipAddressDetails == null) {
- throw new DumpEmptyException("Invalid data dumped");
- }
- }
-}
+++ /dev/null
-/*
- * Copyright (c) 2016 Cisco and/or its affiliates.
- *
- * 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 io.fd.honeycomb.vpp.test.read;
-
-
-import io.fd.honeycomb.translate.util.read.cache.EntityDumpNonEmptyCheck;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.DumpCheckFailedException;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.i.DumpEmptyException;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Generic test for classes implementing {@link EntityDumpNonEmptyCheck}
- *
- * @param <T> implementations of {@link EntityDumpNonEmptyCheck}, that will be tested
- * @param <U> data node handled by {@link EntityDumpNonEmptyCheck}
- */
-public abstract class DumpCheckTest<T extends EntityDumpNonEmptyCheck<U>, U> {
-
- private T check;
- private U emptyData;
- private U validData;
-
- @Before
- public void setupParent() {
- this.check = initCheck();
- this.emptyData = initEmptyData();
- this.validData = initValidData();
- }
-
- @Test(expected = DumpEmptyException.class)
- public void testWithNull() throws DumpCheckFailedException {
- check.assertNotEmpty(null);
- }
-
- @Test(expected = DumpEmptyException.class)
- public void testWithEmpty() throws DumpCheckFailedException {
- check.assertNotEmpty(emptyData);
- }
-
- @Test
- public void testWithValid() throws DumpCheckFailedException {
- check.assertNotEmpty(validData);
- }
-
- /**
- * Initialize new {@link EntityDumpNonEmptyCheck}.
- */
- protected abstract T initCheck();
-
- /**
- * Initialize data that should throw {@link DumpEmptyException} ,
- * while beeing processed by {@link EntityDumpNonEmptyCheck}
- */
- protected abstract U initEmptyData();
-
- /**
- * Initialize data that should pass without exception ,
- * while beeing processed by {@link EntityDumpNonEmptyCheck}
- */
- protected abstract U initValidData();
-
-
- protected T getCheck() {
- return this.check;
- }
-}
package io.fd.honeycomb.translate.vpp.util.cache;
+import static io.fd.honeycomb.translate.util.read.cache.EntityDumpExecutor.NO_PARAMS;
import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.when;
import io.fd.honeycomb.translate.ModificationCache;
import io.fd.honeycomb.translate.util.read.cache.DumpCacheManager;
import io.fd.honeycomb.translate.util.read.cache.EntityDumpExecutor;
-import io.fd.honeycomb.translate.util.read.cache.EntityDumpNonEmptyCheck;
import io.fd.honeycomb.translate.util.read.cache.EntityDumpPostProcessingFunction;
-import io.fd.honeycomb.translate.util.read.cache.exceptions.check.i.DumpEmptyException;
import io.fd.honeycomb.translate.util.read.cache.exceptions.execution.DumpExecutionFailedException;
import org.junit.Before;
import org.junit.Test;
managerPositive =
new DumpCacheManager.DumpCacheManagerBuilder<IpDetailsReplyDump, Void>()
.withExecutor(executor)
- .withNonEmptyPredicate(createPositivePredicate())
.build();
managerPositiveWithPostProcessing = new DumpCacheManager.DumpCacheManagerBuilder<IpDetailsReplyDump, Void>()
.withExecutor(executor)
- .withNonEmptyPredicate(createPositivePredicate())
.withPostProcessingFunction(createPostProcessor())
.build();
managerNegative =
new DumpCacheManager.DumpCacheManagerBuilder<IpDetailsReplyDump, Void>()
.withExecutor(executor)
- .withNonEmptyPredicate(createNegativePredicate())
.build();
cache = new ModificationCache();
*/
@Test
public void testCaching() throws DumpExecutionFailedException {
+ final IpDetailsReplyDump stage1Data = new IpDetailsReplyDump();
+ // executor cant return null data
+ when(executor.executeDump(NO_PARAMS)).thenReturn(new IpDetailsReplyDump());
- Optional<IpDetailsReplyDump> stage1Optional = managerNegative.getDump(KEY, cache, null);
+ final Optional<IpDetailsReplyDump> stage1Optional = managerNegative.getDump(KEY, cache, NO_PARAMS);
- //this is first call so instance should be from executor
- assertEquals(false, stage1Optional.isPresent());
- assertEquals(false, cache.containsKey(KEY));
+ // this is first call so instance should be from executor
+ // and it should be cached after calling executor
+ assertEquals(true, stage1Optional.isPresent());
+ assertEquals(stage1Data, stage1Optional.get());
+ assertEquals(true, cache.containsKey(KEY));
+ assertEquals(stage1Data, cache.get(KEY));
//rebind executor with other data
IpDetailsReplyDump stage2LoadedDump = new IpDetailsReplyDump();
- when(executor.executeDump(null)).thenReturn(stage2LoadedDump);
+ when(executor.executeDump(NO_PARAMS)).thenReturn(stage2LoadedDump);
- Optional<IpDetailsReplyDump> stage2Optional = managerPositive.getDump(KEY, cache, null);
+ final Optional<IpDetailsReplyDump> stage2Optional = managerPositive.getDump(KEY, cache, NO_PARAMS);
assertEquals(true, stage2Optional.isPresent());
assertEquals(stage2LoadedDump, stage2Optional.get());
//rebind executor with other data
IpDetailsReplyDump stage3LoadedDump = new IpDetailsReplyDump();
- when(executor.executeDump(null)).thenReturn(stage3LoadedDump);
+ when(executor.executeDump(NO_PARAMS)).thenReturn(stage3LoadedDump);
- Optional<IpDetailsReplyDump> stage3Optional = managerPositive.getDump(KEY, cache, null);
+ final Optional<IpDetailsReplyDump> stage3Optional = managerPositive.getDump(KEY, cache, NO_PARAMS);
assertEquals(true, stage3Optional.isPresent());
//check if it returns instance cached from previous stage
assertEquals(stage2LoadedDump, stage3Optional.get());
when(executor.executeDump(null)).thenReturn(dump);
- Optional<IpDetailsReplyDump> optionalDump = managerPositiveWithPostProcessing.getDump(KEY, cache, null);
+ Optional<IpDetailsReplyDump> optionalDump = managerPositiveWithPostProcessing.getDump(KEY, cache, NO_PARAMS);
assertEquals(true, optionalDump.isPresent());
assertEquals(1, optionalDump.get().ipDetails.size());
assertEquals(7, optionalDump.get().ipDetails.get(0).swIfIndex);
}
- private EntityDumpNonEmptyCheck<IpDetailsReplyDump> createNegativePredicate() {
- return data -> {
- throw new DumpEmptyException("Empty dump", new IllegalArgumentException());
- };
- }
-
- private EntityDumpNonEmptyCheck<IpDetailsReplyDump> createPositivePredicate() {
- return data -> {
- //DO NOTHING
- };
- }
-
private EntityDumpPostProcessingFunction<IpDetailsReplyDump> createPostProcessor() {
return ipDetailsReplyDump -> {
IpDetailsReplyDump modified = new IpDetailsReplyDump();