You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jclouds.apache.org by na...@apache.org on 2017/02/02 14:34:18 UTC

[1/9] jclouds-labs git commit: Add Packet ComputeAdapter

Repository: jclouds-labs
Updated Branches:
  refs/heads/2.0.x 66bed4b31 -> 5261cba28


Add Packet ComputeAdapter

- add CreateSshKeysThenCreateNodes strategy
- add PacketComputeServiceLiveTest
- add PacketTemplateBuilderLiveTest
- refactor deviceApi.actions into separate operations
- fix Device.State
- remove overriden live test as it is now pushed up to jclouds/jclouds
- improve comments on the adapter


Project: http://git-wip-us.apache.org/repos/asf/jclouds-labs/repo
Commit: http://git-wip-us.apache.org/repos/asf/jclouds-labs/commit/38f0a50d
Tree: http://git-wip-us.apache.org/repos/asf/jclouds-labs/tree/38f0a50d
Diff: http://git-wip-us.apache.org/repos/asf/jclouds-labs/diff/38f0a50d

Branch: refs/heads/2.0.x
Commit: 38f0a50dc0bec474082e6ebb8fd0ae0cd48a1bb4
Parents: cbe306f
Author: Andrea Turli <an...@gmail.com>
Authored: Thu Jan 26 13:09:07 2017 +0100
Committer: Ignasi Barrera <na...@apache.org>
Committed: Thu Feb 2 15:23:25 2017 +0100

----------------------------------------------------------------------
 .../org/jclouds/packet/PacketApiMetadata.java   |   9 +-
 .../compute/PacketComputeServiceAdapter.java    | 184 ++++++++++++++++
 .../PacketComputeServiceContextModule.java      | 151 +++++++++++++
 .../compute/functions/DeviceStateToStatus.java  |   9 +-
 .../compute/functions/DeviceToNodeMetadata.java |   2 +-
 .../compute/options/PacketTemplateOptions.java  | 158 ++++++++++++++
 .../strategy/CreateSshKeysThenCreateNodes.java  | 215 +++++++++++++++++++
 .../java/org/jclouds/packet/domain/Device.java  |   2 +-
 .../org/jclouds/packet/features/DeviceApi.java  |  35 ++-
 .../PacketComputeProviderMetadataTest.java      |  30 +++
 .../compute/PacketComputeServiceLiveTest.java   |  83 +++++++
 .../compute/PacketTemplateBuilderLiveTest.java  |  55 +++++
 .../compute/internal/BasePacketApiLiveTest.java |   8 +
 .../packet/features/DeviceApiLiveTest.java      |  15 +-
 .../packet/features/DeviceApiMockTest.java      |  30 +--
 .../packet/features/FacilityApiLiveTest.java    |   4 +-
 packet/src/test/resources/rescue.json           |   3 -
 17 files changed, 944 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java b/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
index e05685e..5893ea0 100644
--- a/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
+++ b/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
@@ -20,6 +20,8 @@ import java.net.URI;
 import java.util.Properties;
 
 import org.jclouds.apis.ApiMetadata;
+import org.jclouds.compute.ComputeServiceContext;
+import org.jclouds.packet.compute.config.PacketComputeServiceContextModule;
 import org.jclouds.packet.config.PacketComputeParserModule;
 import org.jclouds.packet.config.PacketHttpApiModule;
 import org.jclouds.rest.internal.BaseHttpApiMetadata;
@@ -29,6 +31,8 @@ import com.google.inject.Module;
 
 import static org.jclouds.compute.config.ComputeServiceProperties.TEMPLATE;
 import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_RUNNING;
+import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_SUSPENDED;
+import static org.jclouds.reflect.Reflection2.typeToken;
 
 /**
  * Implementation of {@link ApiMetadata} for Packet API
@@ -52,6 +56,7 @@ public class PacketApiMetadata extends BaseHttpApiMetadata<PacketApi> {
       Properties properties = BaseHttpApiMetadata.defaultProperties();
       properties.put(TEMPLATE, "osFamily=UBUNTU,os64Bit=true,osVersionMatches=16.*");
       properties.put(TIMEOUT_NODE_RUNNING, 300000); // 5 mins
+      properties.put(TIMEOUT_NODE_SUSPENDED, 300000); // 5 mins
       return properties;
    }
 
@@ -66,11 +71,11 @@ public class PacketApiMetadata extends BaseHttpApiMetadata<PacketApi> {
                  .defaultEndpoint("https://api.packet.net")
                  .defaultProperties(PacketApiMetadata.defaultProperties())
                  .version("1")
-                 //.view(typeToken(ComputeServiceContext.class))
+                 .view(typeToken(ComputeServiceContext.class))
                  .defaultModules(ImmutableSet.<Class<? extends Module>>builder()
                          .add(PacketHttpApiModule.class)
                          .add(PacketComputeParserModule.class)
-                         //.add(PacketComputeServiceContextModule.class)
+                         .add(PacketComputeServiceContextModule.class)
                          .build());
       }
 

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/main/java/org/jclouds/packet/compute/PacketComputeServiceAdapter.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/PacketComputeServiceAdapter.java b/packet/src/main/java/org/jclouds/packet/compute/PacketComputeServiceAdapter.java
new file mode 100644
index 0000000..3672b14
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/compute/PacketComputeServiceAdapter.java
@@ -0,0 +1,184 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute;
+
+import java.util.Map;
+import java.util.Set;
+
+import javax.annotation.Resource;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.inject.Singleton;
+
+import org.jclouds.compute.ComputeServiceAdapter;
+import org.jclouds.compute.domain.Template;
+import org.jclouds.compute.reference.ComputeServiceConstants;
+import org.jclouds.domain.Credentials;
+import org.jclouds.location.Provider;
+import org.jclouds.logging.Logger;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.compute.options.PacketTemplateOptions;
+import org.jclouds.packet.domain.BillingCycle;
+import org.jclouds.packet.domain.Device;
+import org.jclouds.packet.domain.Facility;
+import org.jclouds.packet.domain.OperatingSystem;
+import org.jclouds.packet.domain.Plan;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Predicate;
+import com.google.common.base.Supplier;
+import com.google.common.collect.Iterables;
+
+import static com.google.common.collect.Iterables.contains;
+import static com.google.common.collect.Iterables.filter;
+
+/**
+ * defines the connection between the {@link org.jclouds.packet.PacketApi} implementation and
+ * the jclouds {@link org.jclouds.compute.ComputeService}
+ */
+@Singleton
+public class PacketComputeServiceAdapter implements ComputeServiceAdapter<Device, Plan, OperatingSystem, Facility> {
+
+   @Resource
+   @Named(ComputeServiceConstants.COMPUTE_LOGGER)
+   protected Logger logger = Logger.NULL;
+
+   private final PacketApi api;
+   private final String projectId;
+
+   @Inject
+   PacketComputeServiceAdapter(PacketApi api, @Provider final Supplier<Credentials> creds) {
+      this.api = api;
+      this.projectId = creds.get().identity;
+   }
+
+   @Override
+   public NodeAndInitialCredentials<Device> createNodeWithGroupEncodedIntoName(String group, String name, Template template) {
+
+      PacketTemplateOptions templateOptions = template.getOptions().as(PacketTemplateOptions.class);
+      Map<String, String> features = templateOptions.getFeatures();
+      BillingCycle billingCycle = BillingCycle.fromValue(templateOptions.getBillingCycle());
+      boolean locked = templateOptions.isLocked();
+      String userdata = templateOptions.getUserData();
+      Set<String> tags = templateOptions.getTags();
+
+      String plan = template.getHardware().getId();
+      String facility = template.getLocation().getId();
+      String operatingSystem = template.getImage().getId();
+
+      Device device = api.deviceApi(projectId).create(
+              Device.CreateDevice.builder()
+                      .hostname(name)
+                      .plan(plan)
+                      .billingCycle(billingCycle.value())
+                      .facility(facility)
+                      .features(features)
+                      .operatingSystem(operatingSystem)
+                      .locked(locked)
+                      .userdata(userdata)
+                      .tags(tags)
+                      .build()
+              );
+
+      // Any new servers you deploy to projects you are a collaborator on will have your project and personal SSH keys, if defined.
+      // If no SSH keys are defined in your account, jclouds will generate one usiing CreateSshKeysThenCreateNodes 
+      // so that it will add it to the device with the default mechanism.
+
+      // Safe to pass null credentials here, as jclouds will default populate
+      // the node with the default credentials from the image, or the ones in
+      // the options, if provided.
+      return new NodeAndInitialCredentials<Device>(device, device.id(), null);
+   }
+
+   @Override
+   public Iterable<Plan> listHardwareProfiles() {
+      return Iterables.filter(api.planApi().list().concat(), new Predicate<Plan>() {
+         @Override
+         public boolean apply(Plan input) {
+            return input.line().equals("baremetal");
+         }
+      });
+   }
+
+   @Override
+   public Iterable<OperatingSystem> listImages() {
+      return api.operatingSystemApi().list().concat();
+   }
+
+   @Override
+   public OperatingSystem getImage(final String id) {
+      Optional<OperatingSystem> firstInterestingOperatingSystem = api
+              .operatingSystemApi().list()
+              .concat()
+              .firstMatch(new Predicate<OperatingSystem>() {
+                 @Override
+                 public boolean apply(OperatingSystem input) {
+                    return input.slug().equals(id);
+                 }
+              });
+      if (!firstInterestingOperatingSystem.isPresent()) {
+         throw new IllegalStateException("Cannot find image with the required slug " + id);
+      }
+      return firstInterestingOperatingSystem.get();
+   }
+
+   @Override
+   public Iterable<Facility> listLocations() {
+      return api.facilityApi().list().concat();
+   }
+
+   @Override
+   public Device getNode(String id) {
+      return api.deviceApi(projectId).get(id);
+   }
+
+   @Override
+   public void destroyNode(String id) {
+      api.deviceApi(projectId).delete(id);
+   }
+
+   @Override
+   public void rebootNode(String id) {
+      api.deviceApi(projectId).reboot(id);
+   }
+
+   @Override
+   public void resumeNode(String id) {
+      api.deviceApi(projectId).powerOn(id);
+   }
+
+   @Override
+   public void suspendNode(String id) {
+      api.deviceApi(projectId).powerOff(id);
+   }
+
+   @Override
+   public Iterable<Device> listNodes() {
+     return api.deviceApi(projectId).list().concat();
+   }
+
+   @Override
+   public Iterable<Device> listNodesByIds(final Iterable<String> ids) {
+      return filter(listNodes(), new Predicate<Device>() {
+         @Override
+         public boolean apply(Device device) {
+            return contains(ids, String.valueOf(device.id()));
+         }
+      });
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/main/java/org/jclouds/packet/compute/config/PacketComputeServiceContextModule.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/config/PacketComputeServiceContextModule.java b/packet/src/main/java/org/jclouds/packet/compute/config/PacketComputeServiceContextModule.java
new file mode 100644
index 0000000..0a64f43
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/compute/config/PacketComputeServiceContextModule.java
@@ -0,0 +1,151 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.config;
+
+import org.jclouds.compute.ComputeServiceAdapter;
+import org.jclouds.compute.config.ComputeServiceAdapterContextModule;
+import org.jclouds.compute.domain.Hardware;
+import org.jclouds.compute.domain.Image;
+import org.jclouds.compute.domain.NodeMetadata;
+import org.jclouds.compute.options.TemplateOptions;
+import org.jclouds.compute.reference.ComputeServiceConstants;
+import org.jclouds.compute.strategy.CreateNodesInGroupThenAddToSet;
+import org.jclouds.domain.Credentials;
+import org.jclouds.domain.Location;
+import org.jclouds.location.Provider;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.compute.PacketComputeServiceAdapter;
+import org.jclouds.packet.compute.functions.DeviceStateToStatus;
+import org.jclouds.packet.compute.functions.DeviceToNodeMetadata;
+import org.jclouds.packet.compute.functions.FacilityToLocation;
+import org.jclouds.packet.compute.functions.OperatingSystemToImage;
+import org.jclouds.packet.compute.functions.PlanToHardware;
+import org.jclouds.packet.compute.options.PacketTemplateOptions;
+import org.jclouds.packet.compute.strategy.CreateSshKeysThenCreateNodes;
+import org.jclouds.packet.domain.Device;
+import org.jclouds.packet.domain.Facility;
+import org.jclouds.packet.domain.OperatingSystem;
+import org.jclouds.packet.domain.Plan;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.base.Supplier;
+import com.google.inject.Provides;
+import com.google.inject.TypeLiteral;
+import com.google.inject.name.Named;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_RUNNING;
+import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_SUSPENDED;
+import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_TERMINATED;
+import static org.jclouds.util.Predicates2.retry;
+
+public class PacketComputeServiceContextModule extends
+        ComputeServiceAdapterContextModule<Device, Plan, OperatingSystem, Facility> {
+
+   @SuppressWarnings("unchecked")
+   @Override
+   protected void configure() {
+      super.configure();
+
+      bind(new TypeLiteral<ComputeServiceAdapter<Device, Plan, OperatingSystem, Facility>>() {
+      }).to(PacketComputeServiceAdapter.class);
+
+      bind(new TypeLiteral<Function<Device, NodeMetadata>>() {
+      }).to(DeviceToNodeMetadata.class);
+      bind(new TypeLiteral<Function<Plan, Hardware>>() {
+      }).to(PlanToHardware.class);
+      bind(new TypeLiteral<Function<OperatingSystem, Image>>() {
+      }).to(OperatingSystemToImage.class);
+      bind(new TypeLiteral<Function<Facility, Location>>() {
+      }).to(FacilityToLocation.class);
+      bind(new TypeLiteral<Function<Device.State, NodeMetadata.Status>>() {
+      }).to(DeviceStateToStatus.class);
+      install(new LocationsFromComputeServiceAdapterModule<Device, Plan, OperatingSystem, Facility>() {
+      });
+      bind(TemplateOptions.class).to(PacketTemplateOptions.class);
+      bind(CreateNodesInGroupThenAddToSet.class).to(CreateSshKeysThenCreateNodes.class);
+   }
+
+   @Provides
+   @Named(TIMEOUT_NODE_RUNNING)
+   protected Predicate<String> provideDeviceRunningPredicate(final PacketApi api,
+                                                             @Provider final Supplier<Credentials> creds,
+                                                             ComputeServiceConstants.Timeouts timeouts,
+                                                             ComputeServiceConstants.PollPeriod pollPeriod) {
+      return retry(new DeviceInStatusPredicate(api, creds.get().identity, Device.State.ACTIVE), timeouts.nodeRunning,
+              pollPeriod.pollInitialPeriod, pollPeriod.pollMaxPeriod);
+   }
+
+   @Provides
+   @Named(TIMEOUT_NODE_SUSPENDED)
+   protected Predicate<String> provideDeviceSuspendedPredicate(final PacketApi api, @Provider final Supplier<Credentials> creds, ComputeServiceConstants.Timeouts timeouts,
+                                                                 ComputeServiceConstants.PollPeriod pollPeriod) {
+      return retry(new DeviceInStatusPredicate(api, creds.get().identity, Device.State.INACTIVE), timeouts.nodeSuspended,
+              pollPeriod.pollInitialPeriod, pollPeriod.pollMaxPeriod);
+   }
+   
+   @Provides
+   @Named(TIMEOUT_NODE_TERMINATED)
+   protected Predicate<String> provideDeviceTerminatedPredicate(final PacketApi api, @Provider final Supplier<Credentials> creds, ComputeServiceConstants.Timeouts timeouts,
+                                                                 ComputeServiceConstants.PollPeriod pollPeriod) {
+      return retry(new DeviceTerminatedPredicate(api, creds.get().identity), timeouts.nodeTerminated, pollPeriod.pollInitialPeriod,
+              pollPeriod.pollMaxPeriod);
+   }
+
+   @VisibleForTesting
+   static class DeviceInStatusPredicate implements Predicate<String> {
+
+      private final PacketApi api;
+      private final String projectId;
+      private final Device.State state;
+
+      public DeviceInStatusPredicate(PacketApi api, String projectId, Device.State state) {
+         this.api = checkNotNull(api, "api must not be null");
+         this.projectId = checkNotNull(projectId, "projectId must not be null");
+         this.state = checkNotNull(state, "state must not be null");
+      }
+
+      @Override
+      public boolean apply(String input) {
+         checkNotNull(input, "device id");
+         Device device = api.deviceApi(projectId).get(input);
+         return device != null && state == device.state();
+      }
+   }
+
+   @VisibleForTesting
+   static class DeviceTerminatedPredicate implements Predicate<String> {
+
+      private final PacketApi api;
+      private final String projectId;
+
+      public DeviceTerminatedPredicate(PacketApi api, String projectId) {
+         this.api = checkNotNull(api, "api must not be null");
+         this.projectId = checkNotNull(projectId, "projectId must not be null");
+      }
+
+      @Override
+      public boolean apply(String input) {
+         checkNotNull(input, "device id");
+         Device device = api.deviceApi(projectId).get(input);
+         return device == null;
+      }
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceStateToStatus.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceStateToStatus.java b/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceStateToStatus.java
index acb1f00..7606bf5 100644
--- a/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceStateToStatus.java
+++ b/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceStateToStatus.java
@@ -33,8 +33,13 @@ public class DeviceStateToStatus implements Function<Device.State, Status> {
 
    private static final Function<Device.State, Status> toPortableStatus = Functions.forMap(
          ImmutableMap.<Device.State, Status> builder()
-               .put(Device.State.PROVISIONING, Status.PENDING)
-               .put(Device.State.ACTIVE, Status.RUNNING)
+                 .put(Device.State.PROVISIONING, Status.PENDING)
+                 .put(Device.State.POWERING_ON, Status.PENDING)
+                 .put(Device.State.POWERING_OFF, Status.PENDING)
+                 .put(Device.State.REBOOTING, Status.PENDING)
+                 .put(Device.State.QUEUED, Status.PENDING)
+                 .put(Device.State.INACTIVE, Status.SUSPENDED)
+                 .put(Device.State.ACTIVE, Status.RUNNING)
                .build(),
          Status.UNRECOGNIZED);
 

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceToNodeMetadata.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceToNodeMetadata.java b/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceToNodeMetadata.java
index 74d9d95..ec222fc 100644
--- a/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceToNodeMetadata.java
+++ b/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceToNodeMetadata.java
@@ -69,7 +69,7 @@ public class DeviceToNodeMetadata implements Function<Device, NodeMetadata> {
       return new NodeMetadataBuilder()
               .ids(input.id())
               .name(input.hostname())
-              .hostname(input.hostname())
+              .hostname(String.format("%s", input.hostname()))  
               .group(groupNamingConvention.extractGroup(input.hostname()))
               .location(facilityToLocation.apply(input.facility()))
               .hardware(planToHardware.apply(input.plan()))

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/main/java/org/jclouds/packet/compute/options/PacketTemplateOptions.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/options/PacketTemplateOptions.java b/packet/src/main/java/org/jclouds/packet/compute/options/PacketTemplateOptions.java
new file mode 100644
index 0000000..30cef45
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/compute/options/PacketTemplateOptions.java
@@ -0,0 +1,158 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.options;
+
+import java.util.Map;
+
+import org.jclouds.compute.options.TemplateOptions;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Objects.ToStringHelper;
+import com.google.common.collect.ImmutableMap;
+
+import static com.google.common.base.Objects.equal;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Custom options for the Packet API.
+ */
+public class PacketTemplateOptions extends TemplateOptions implements Cloneable {
+
+   private Map<String, String> features = ImmutableMap.of();
+   private boolean locked = false;
+   private String billingCycle = "hourly";
+   private String userData = "";
+
+   public PacketTemplateOptions features(Map<String, String> features) {
+      this.features = ImmutableMap.copyOf(checkNotNull(features, "features cannot be null"));
+      return this;
+   }
+   
+   public PacketTemplateOptions locked(boolean locked) {
+      this.locked = locked;
+      return this;
+   }
+
+   public PacketTemplateOptions billingCycle(String billingCycle) {
+      this.billingCycle = billingCycle;
+      return this;
+   }
+   
+   public PacketTemplateOptions userData(String userData) {
+      this.userData = userData;
+      return this;
+   }
+
+   public Map<String, String> getFeatures() {
+      return features;
+   }
+   public boolean isLocked() {
+      return locked;
+   }
+   public String getBillingCycle() {
+      return billingCycle;
+   }
+   public String getUserData() {
+      return userData;
+   }
+
+   @Override
+   public PacketTemplateOptions clone() {
+      PacketTemplateOptions options = new PacketTemplateOptions();
+      copyTo(options);
+      return options;
+   }
+
+   @Override
+   public void copyTo(TemplateOptions to) {
+      super.copyTo(to);
+      if (to instanceof PacketTemplateOptions) {
+         PacketTemplateOptions eTo = PacketTemplateOptions.class.cast(to);
+         eTo.features(features);
+         eTo.locked(locked);
+         eTo.billingCycle(billingCycle);
+         eTo.userData(userData);
+      }
+   }
+
+   @Override
+   public int hashCode() {
+      return Objects.hashCode(super.hashCode(), features, locked, billingCycle, userData);
+   }
+
+   @Override
+   public boolean equals(Object obj) {
+      if (this == obj) {
+         return true;
+      }
+      if (!super.equals(obj)) {
+         return false;
+      }
+      if (getClass() != obj.getClass()) {
+         return false;
+      }
+      PacketTemplateOptions other = (PacketTemplateOptions) obj;
+      return super.equals(other) && equal(this.locked, other.locked) && equal(this.billingCycle, other.billingCycle) && equal(this.userData, other.userData) && equal(this.features, other.features);
+   }
+
+   @Override
+   public ToStringHelper string() {
+      ToStringHelper toString = super.string().omitNullValues();
+      if (!features.isEmpty()) {
+         toString.add("features", features);
+      }      toString.add("locked", locked);
+      toString.add("billingCycle", billingCycle);
+      toString.add("userData", userData);
+      return toString;
+   }
+
+   public static class Builder {
+
+      /**
+       * @see PacketTemplateOptions#features
+       */
+      public static PacketTemplateOptions features(Map<String, String> features) {
+         PacketTemplateOptions options = new PacketTemplateOptions();
+         return options.features(features);
+      }
+      
+      /**
+       * @see PacketTemplateOptions#locked
+       */
+      public static PacketTemplateOptions locked(boolean locked) {
+         PacketTemplateOptions options = new PacketTemplateOptions();
+         return options.locked(locked);
+      }
+
+      /**
+       * @see PacketTemplateOptions#billingCycle
+       */
+      public static PacketTemplateOptions billingCycle(String billingCycle) {
+         PacketTemplateOptions options = new PacketTemplateOptions();
+         return options.billingCycle(billingCycle);
+      }
+      
+      /**
+       * @see PacketTemplateOptions#userData
+       */
+      public static PacketTemplateOptions userData(String userData) {
+         PacketTemplateOptions options = new PacketTemplateOptions();
+         return options.userData(userData);
+      }
+
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/main/java/org/jclouds/packet/compute/strategy/CreateSshKeysThenCreateNodes.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/strategy/CreateSshKeysThenCreateNodes.java b/packet/src/main/java/org/jclouds/packet/compute/strategy/CreateSshKeysThenCreateNodes.java
new file mode 100644
index 0000000..3a5a02a
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/compute/strategy/CreateSshKeysThenCreateNodes.java
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.strategy;
+
+import java.security.KeyFactory;
+import java.security.NoSuchAlgorithmException;
+import java.security.PublicKey;
+import java.security.interfaces.RSAPublicKey;
+import java.security.spec.InvalidKeySpecException;
+import java.security.spec.RSAPublicKeySpec;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.annotation.Resource;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.inject.Singleton;
+
+import org.jclouds.Constants;
+import org.jclouds.compute.config.CustomizationResponse;
+import org.jclouds.compute.domain.NodeMetadata;
+import org.jclouds.compute.domain.Template;
+import org.jclouds.compute.functions.GroupNamingConvention;
+import org.jclouds.compute.reference.ComputeServiceConstants;
+import org.jclouds.compute.strategy.CreateNodeWithGroupEncodedIntoName;
+import org.jclouds.compute.strategy.CustomizeNodeAndAddToGoodMapOrPutExceptionIntoBadMap;
+import org.jclouds.compute.strategy.ListNodesStrategy;
+import org.jclouds.compute.strategy.impl.CreateNodesWithGroupEncodedIntoNameThenAddToSet;
+import org.jclouds.logging.Logger;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.compute.options.PacketTemplateOptions;
+import org.jclouds.packet.domain.SshKey;
+import org.jclouds.ssh.SshKeyPairGenerator;
+import org.jclouds.ssh.SshKeys;
+
+import com.google.common.base.Splitter;
+import com.google.common.base.Strings;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Throwables.propagate;
+import static com.google.common.collect.Iterables.get;
+import static com.google.common.collect.Iterables.size;
+
+@Singleton
+public class CreateSshKeysThenCreateNodes extends CreateNodesWithGroupEncodedIntoNameThenAddToSet {
+
+    @Resource
+    @Named(ComputeServiceConstants.COMPUTE_LOGGER)
+    protected Logger logger = Logger.NULL;
+
+    private final PacketApi api;
+    private final SshKeyPairGenerator keyGenerator;
+
+    @Inject
+    protected CreateSshKeysThenCreateNodes(
+            CreateNodeWithGroupEncodedIntoName addNodeWithGroupStrategy,
+            ListNodesStrategy listNodesStrategy,
+            GroupNamingConvention.Factory namingConvention,
+            @Named(Constants.PROPERTY_USER_THREADS) ListeningExecutorService userExecutor,
+            CustomizeNodeAndAddToGoodMapOrPutExceptionIntoBadMap.Factory customizeNodeAndAddToGoodMapOrPutExceptionIntoBadMapFactory,
+            PacketApi api, SshKeyPairGenerator keyGenerator) {
+        super(addNodeWithGroupStrategy, listNodesStrategy, namingConvention, userExecutor,
+                customizeNodeAndAddToGoodMapOrPutExceptionIntoBadMapFactory);
+        this.api = api;
+        this.keyGenerator = keyGenerator;
+    }
+
+    @Override
+    public Map<?, ListenableFuture<Void>> execute(String group, int count, Template template,
+                                                  Set<NodeMetadata> goodNodes, Map<NodeMetadata, Exception> badNodes,
+                                                  Multimap<NodeMetadata, CustomizationResponse> customizationResponses) {
+
+        PacketTemplateOptions options = template.getOptions().as(PacketTemplateOptions.class);
+        Set<String> generatedSshKeyIds = Sets.newHashSet();
+
+        // If no key has been configured, generate a key pair
+        if (Strings.isNullOrEmpty(options.getPublicKey())) {
+            generateKeyPairAndAddKeyToSet(options, generatedSshKeyIds, group);
+        }
+
+        // If there is a script to run in the node, make sure a private key has
+        // been configured so jclouds will be able to access the node
+        if (options.getRunScript() != null && Strings.isNullOrEmpty(options.getLoginPrivateKey())) {
+            logger.warn(">> A runScript has been configured but no SSH key has been provided."
+                    + " Authentication will delegate to the ssh-agent");
+        }
+
+        // If there is a key configured, then make sure there is a key pair for it
+        if (!Strings.isNullOrEmpty(options.getPublicKey())) {
+            createKeyPairForPublicKeyInOptionsAndAddToSet(options, generatedSshKeyIds);
+        }
+
+        Map<?, ListenableFuture<Void>> responses = super.execute(group, count, template, goodNodes, badNodes,
+                customizationResponses);
+
+        // Key pairs in Packet are only required to create the devices. They aren't used anymore so it is better
+        // to delete the auto-generated key pairs at this point where we know exactly which ones have been
+        // auto-generated by jclouds.
+        registerAutoGeneratedKeyPairCleanupCallbacks(responses, generatedSshKeyIds);
+
+        return responses;
+    }
+
+    private void createKeyPairForPublicKeyInOptionsAndAddToSet(PacketTemplateOptions options,
+                                                               Set<String> generatedSshKeyIds) {
+        logger.debug(">> checking if the key pair already exists...");
+
+        PublicKey userKey;
+        Iterable<String> parts = Splitter.on(' ').split(options.getPublicKey());
+        checkArgument(size(parts) >= 2, "bad format, should be: ssh-rsa AAAAB3...");
+        String type = get(parts, 0);
+
+        try {
+            if ("ssh-rsa".equals(type)) {
+                RSAPublicKeySpec spec = SshKeys.publicKeySpecFromOpenSSH(options.getPublicKey());
+                userKey = KeyFactory.getInstance("RSA").generatePublic(spec);
+            } else {
+                throw new IllegalArgumentException("bad format, ssh-rsa is only supported");
+            }
+        } catch (InvalidKeySpecException ex) {
+            throw propagate(ex);
+        } catch (NoSuchAlgorithmException ex) {
+            throw propagate(ex);
+        }
+      String label = computeFingerprint(userKey);
+      SshKey key = api.sshKeyApi().get(label);
+
+      if (key == null) {
+         logger.debug(">> key pair not found. creating a new key pair %s ...", label);
+         SshKey newKey = api.sshKeyApi().create(label, options.getPublicKey());
+         logger.debug(">> key pair created! %s", newKey);
+      } else {
+         logger.debug(">> key pair found! %s", key);
+         generatedSshKeyIds.add(key.id());
+      }
+    }
+
+    private void generateKeyPairAndAddKeyToSet(PacketTemplateOptions options, Set<String> generatedSshKeyIds, String prefix) {
+        logger.debug(">> creating default keypair for node...");
+
+        Map<String, String> defaultKeys = keyGenerator.get();
+
+        SshKey sshKey = api.sshKeyApi().create(prefix + System.getProperty("user.name"), defaultKeys.get("public"));
+        generatedSshKeyIds.add(sshKey.id());
+        logger.debug(">> keypair created! %s", sshKey);
+
+        // If a private key has not been explicitly set, configure the generated one
+        if (Strings.isNullOrEmpty(options.getLoginPrivateKey())) {
+            options.overrideLoginPrivateKey(defaultKeys.get("private"));
+        }
+    }
+
+    private void registerAutoGeneratedKeyPairCleanupCallbacks(Map<?, ListenableFuture<Void>> responses,
+                                                              final Set<String> generatedSshKeyIds) {
+        // The Futures.allAsList fails immediately if some of the futures fail. The Futures.successfulAsList, however,
+        // returns a list containing the results or 'null' for those futures that failed. We want to wait for all them
+        // (even if they fail), so better use the latter form.
+        ListenableFuture<List<Void>> aggregatedResponses = Futures.successfulAsList(responses.values());
+
+        // Key pairs must be cleaned up after all futures completed (even if some failed).
+        Futures.addCallback(aggregatedResponses, new FutureCallback<List<Void>>() {
+            @Override
+            public void onSuccess(List<Void> result) {
+                cleanupAutoGeneratedKeyPairs(generatedSshKeyIds);
+            }
+
+            @Override
+            public void onFailure(Throwable t) {
+                cleanupAutoGeneratedKeyPairs(generatedSshKeyIds);
+            }
+
+            private void cleanupAutoGeneratedKeyPairs(Set<String> generatedSshKeyIds) {
+                logger.debug(">> cleaning up auto-generated key pairs...");
+                for (String sshKeyId : generatedSshKeyIds) {
+                    try {
+                        api.sshKeyApi().delete(sshKeyId);
+                    } catch (Exception ex) {
+                        logger.warn(">> could not delete key pair %s: %s", sshKeyId, ex.getMessage());
+                    }
+                }
+            }
+        }, userExecutor);
+    }
+
+    private static String computeFingerprint(PublicKey key) {
+        if (key instanceof RSAPublicKey) {
+            RSAPublicKey rsaKey = (RSAPublicKey) key;
+            return SshKeys.fingerprint(rsaKey.getPublicExponent(), rsaKey.getModulus());
+        } else {
+            throw new IllegalArgumentException("Only RSA keys are supported");
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/main/java/org/jclouds/packet/domain/Device.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Device.java b/packet/src/main/java/org/jclouds/packet/domain/Device.java
index 951d938..18f5d95 100644
--- a/packet/src/main/java/org/jclouds/packet/domain/Device.java
+++ b/packet/src/main/java/org/jclouds/packet/domain/Device.java
@@ -38,7 +38,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 public abstract class Device {
 
     public enum State {
-        PROVISIONING, QUEUED, ACTIVE;
+        PROVISIONING, QUEUED, ACTIVE, REBOOTING, POWERING_OFF, POWERING_ON, INACTIVE;
 
         public static State fromValue(String value) {
             Optional<State> state = Enums.getIfPresent(State.class, value.toUpperCase());

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/main/java/org/jclouds/packet/features/DeviceApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/features/DeviceApi.java b/packet/src/main/java/org/jclouds/packet/features/DeviceApi.java
index 9f4c672..cae9305 100644
--- a/packet/src/main/java/org/jclouds/packet/features/DeviceApi.java
+++ b/packet/src/main/java/org/jclouds/packet/features/DeviceApi.java
@@ -40,7 +40,6 @@ import org.jclouds.http.functions.ParseJson;
 import org.jclouds.javax.annotation.Nullable;
 import org.jclouds.json.Json;
 import org.jclouds.packet.PacketApi;
-import org.jclouds.packet.domain.ActionType;
 import org.jclouds.packet.domain.Device;
 import org.jclouds.packet.domain.Href;
 import org.jclouds.packet.domain.internal.PaginatedCollection;
@@ -49,8 +48,7 @@ import org.jclouds.packet.filters.AddApiVersionToRequest;
 import org.jclouds.packet.filters.AddXAuthTokenToRequest;
 import org.jclouds.rest.annotations.BinderParam;
 import org.jclouds.rest.annotations.Fallback;
-import org.jclouds.rest.annotations.MapBinder;
-import org.jclouds.rest.annotations.PayloadParam;
+import org.jclouds.rest.annotations.Payload;
 import org.jclouds.rest.annotations.RequestFilters;
 import org.jclouds.rest.annotations.ResponseParser;
 import org.jclouds.rest.annotations.Transform;
@@ -60,13 +58,13 @@ import com.google.common.base.Function;
 import com.google.common.base.Optional;
 import com.google.inject.TypeLiteral;
 
-@Path("/projects/{projectId}/devices")
 @Consumes(MediaType.APPLICATION_JSON)
 @RequestFilters({AddXAuthTokenToRequest.class, AddApiVersionToRequest.class})
 public interface DeviceApi {
 
    @Named("device:list")
    @GET
+   @Path("/projects/{projectId}/devices")
    @ResponseParser(ParseDevices.class)
    @Transform(ParseDevices.ToPagedIterable.class)
    @Fallback(Fallbacks.EmptyPagedIterableOnNotFoundOr404.class)
@@ -74,6 +72,7 @@ public interface DeviceApi {
 
    @Named("device:list")
    @GET
+   @Path("/projects/{projectId}/devices")
    @ResponseParser(ParseDevices.class)
    @Fallback(Fallbacks.EmptyIterableWithMarkerOnNotFoundOr404.class)
    IterableWithMarker<Device> list(ListOptions options);
@@ -122,27 +121,43 @@ public interface DeviceApi {
 
    @Named("device:create")
    @POST
+   @Path("/projects/{projectId}/devices")
    @Produces(MediaType.APPLICATION_JSON)
    Device create(@BinderParam(BindToJsonPayload.class) Device.CreateDevice device);
 
 
    @Named("device:get")
    @GET
-   @Path("/{id}")
+   @Path("/devices/{id}")
    @Fallback(NullOnNotFoundOr404.class)
    @Nullable
    Device get(@PathParam("id") String id);
 
    @Named("device:delete")
    @DELETE
-   @Path("/{id}")
+   @Path("/devices/{id}")
    @Fallback(VoidOnNotFoundOr404.class)
    void delete(@PathParam("id") String id);
 
-   @Named("device:actions")
+   @Named("device:powerOff")
    @POST
-   @Path("/{id}/actions")
-   @MapBinder(BindToJsonPayload.class)
-   void actions(@PathParam("id") String id, @PayloadParam("type") ActionType type);
+   @Produces(MediaType.APPLICATION_JSON)
+   @Path("/devices/{id}/actions")
+   @Payload("{\"type\":\"power_off\"}")
+   void powerOff(@PathParam("id") String id);
+
+   @Named("device:powerOn")
+   @POST
+   @Produces(MediaType.APPLICATION_JSON)
+   @Path("/devices/{id}/actions")
+   @Payload("{\"type\":\"power_on\"}")
+   void powerOn(@PathParam("id") String id);
+   
+   @Named("device:reboot")
+   @POST
+   @Produces(MediaType.APPLICATION_JSON)
+   @Path("/devices/{id}/actions")
+   @Payload("{\"type\":\"reboot\"}")
+   void reboot(@PathParam("id") String id);
 
 }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/test/java/org/jclouds/packet/compute/PacketComputeProviderMetadataTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/compute/PacketComputeProviderMetadataTest.java b/packet/src/test/java/org/jclouds/packet/compute/PacketComputeProviderMetadataTest.java
new file mode 100644
index 0000000..a7f0c97
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/compute/PacketComputeProviderMetadataTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute;
+
+import org.jclouds.packet.PacketApiMetadata;
+import org.jclouds.packet.PacketProviderMetadata;
+import org.jclouds.providers.internal.BaseProviderMetadataTest;
+import org.testng.annotations.Test;
+
+@Test(groups = "unit", testName = "PacketComputeProviderMetadataTest")
+public class PacketComputeProviderMetadataTest extends BaseProviderMetadataTest {
+
+   public PacketComputeProviderMetadataTest() {
+      super(new PacketProviderMetadata(), new PacketApiMetadata());
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/test/java/org/jclouds/packet/compute/PacketComputeServiceLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/compute/PacketComputeServiceLiveTest.java b/packet/src/test/java/org/jclouds/packet/compute/PacketComputeServiceLiveTest.java
new file mode 100644
index 0000000..dc10ecd
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/compute/PacketComputeServiceLiveTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute;
+
+import java.util.Properties;
+
+import org.jclouds.compute.ComputeServiceContext;
+import org.jclouds.compute.domain.NodeMetadata;
+import org.jclouds.compute.internal.BaseComputeServiceLiveTest;
+import org.jclouds.rest.AuthorizationException;
+import org.jclouds.sshj.config.SshjSshClientModule;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.inject.Module;
+
+/**
+ * Live tests for the {@link org.jclouds.compute.ComputeService} integration.
+ */
+@Test(groups = "live", singleThreaded = true, testName = "PacketComputeServiceLiveTest")
+public class PacketComputeServiceLiveTest extends BaseComputeServiceLiveTest {
+
+   public PacketComputeServiceLiveTest() {
+      provider = "packet";
+   }
+
+   @Override
+   protected Module getSshModule() {
+      return new SshjSshClientModule();
+   }
+
+   @Override
+   @Test(expectedExceptions = AuthorizationException.class)
+   public void testCorrectAuthException() throws Exception {
+      ComputeServiceContext context = null;
+      try {
+         Properties overrides = setupProperties();
+         overrides.setProperty(provider + ".identity", "MOM:MA");
+         overrides.setProperty(provider + ".credential", "MIA");
+         context = newBuilder()
+                 .modules(ImmutableSet.of(getLoggingModule(), credentialStoreModule))
+                 .overrides(overrides)
+                 .build(ComputeServiceContext.class);
+         // replace listNodes with listImages as it doesn't require `projectId`
+         context.getComputeService().listImages();
+      } catch (AuthorizationException e) {
+         throw e;
+      } catch (RuntimeException e) {
+         e.printStackTrace();
+         throw e;
+      } finally {
+         if (context != null)
+            context.close();
+      }
+   }
+
+   @Override
+   public void testOptionToNotBlock() throws Exception {
+      // Packet ComputeService implementation has to block until the node
+      // is provisioned, to be able to return it.
+   }
+
+   @Override
+   protected void checkUserMetadataContains(NodeMetadata node, ImmutableMap<String, String> userMetadata) {
+      // The Packet API does not return the user data
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/test/java/org/jclouds/packet/compute/PacketTemplateBuilderLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/compute/PacketTemplateBuilderLiveTest.java b/packet/src/test/java/org/jclouds/packet/compute/PacketTemplateBuilderLiveTest.java
new file mode 100644
index 0000000..5faddde
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/compute/PacketTemplateBuilderLiveTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.jclouds.compute.domain.OsFamily;
+import org.jclouds.compute.domain.Template;
+import org.jclouds.compute.internal.BaseTemplateBuilderLiveTest;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableSet;
+
+import static org.jclouds.compute.util.ComputeServiceUtils.getCores;
+import static org.testng.Assert.assertEquals;
+
+@Test(groups = "live", testName = "PacketTemplateBuilderLiveTest")
+public class PacketTemplateBuilderLiveTest extends BaseTemplateBuilderLiveTest {
+
+   public PacketTemplateBuilderLiveTest() {
+      provider = "packet";
+   }
+
+   @Test
+   @Override
+   public void testDefaultTemplateBuilder() throws IOException {
+      Template defaultTemplate = view.getComputeService().templateBuilder().build();
+      assert defaultTemplate.getImage().getOperatingSystem().getVersion().startsWith("16.") : defaultTemplate
+            .getImage().getOperatingSystem().getVersion();
+      assertEquals(defaultTemplate.getImage().getOperatingSystem().is64Bit(), true);
+      assertEquals(defaultTemplate.getImage().getOperatingSystem().getFamily(), OsFamily.UBUNTU);
+      assertEquals(getCores(defaultTemplate.getHardware()), 1.0d);
+   }
+
+   @Override
+   protected Set<String> getIso3166Codes() {
+      return ImmutableSet.of("US-CA", "US-NJ", "NL", "JP");
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiLiveTest.java
index 6c8cf63..aacf945 100644
--- a/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiLiveTest.java
+++ b/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiLiveTest.java
@@ -31,12 +31,14 @@ import com.google.inject.TypeLiteral;
 import com.google.inject.name.Names;
 
 import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_RUNNING;
+import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_SUSPENDED;
 import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_TERMINATED;
 import static org.testng.Assert.assertTrue;
 
 public class BasePacketApiLiveTest extends BaseApiLiveTest<PacketApi> {
 
    private Predicate<String> deviceRunning;
+   private Predicate<String> deviceSuspended;
    private Predicate<String> deviceTerminated;
 
    public BasePacketApiLiveTest() {
@@ -57,6 +59,8 @@ public class BasePacketApiLiveTest extends BaseApiLiveTest<PacketApi> {
       Injector injector = newBuilder().modules(modules).overrides(props).buildInjector();
       deviceRunning = injector.getInstance(Key.get(new TypeLiteral<Predicate<String>>(){},
             Names.named(TIMEOUT_NODE_RUNNING)));
+      deviceSuspended = injector.getInstance(Key.get(new TypeLiteral<Predicate<String>>(){},
+              Names.named(TIMEOUT_NODE_SUSPENDED)));
       deviceTerminated = injector.getInstance(Key.get(new TypeLiteral<Predicate<String>>(){},
               Names.named(TIMEOUT_NODE_TERMINATED)));
       return injector.getInstance(PacketApi.class);
@@ -66,6 +70,10 @@ public class BasePacketApiLiveTest extends BaseApiLiveTest<PacketApi> {
       assertTrue(deviceRunning.apply(deviceId), String.format("Device %s did not start in the configured timeout", deviceId));
    }
 
+   protected void assertNodeSuspended(String deviceId) {
+      assertTrue(deviceSuspended.apply(deviceId), String.format("Device %s was not suspended in the configured timeout", deviceId));
+   }
+   
    protected void assertNodeTerminated(String deviceId) {
       assertTrue(deviceTerminated.apply(deviceId), String.format("Device %s was not terminated in the configured timeout", deviceId));
    }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/test/java/org/jclouds/packet/features/DeviceApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/DeviceApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/features/DeviceApiLiveTest.java
index 36c08f1..cc96ebd 100644
--- a/packet/src/test/java/org/jclouds/packet/features/DeviceApiLiveTest.java
+++ b/packet/src/test/java/org/jclouds/packet/features/DeviceApiLiveTest.java
@@ -20,7 +20,6 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.jclouds.packet.compute.internal.BasePacketApiLiveTest;
-import org.jclouds.packet.domain.ActionType;
 import org.jclouds.packet.domain.BillingCycle;
 import org.jclouds.packet.domain.Device;
 import org.jclouds.packet.domain.SshKey;
@@ -40,7 +39,7 @@ import static org.testng.Assert.assertNull;
 import static org.testng.Assert.assertTrue;
 import static org.testng.util.Strings.isNullOrEmpty;
 
-@Test(groups = "live", testName = "DeviceApiLiveTest")
+@Test(groups = "live", singleThreaded = true, testName = "DeviceApiLiveTest")
 public class DeviceApiLiveTest extends BasePacketApiLiveTest {
 
    private SshKey sshKey;
@@ -81,22 +80,22 @@ public class DeviceApiLiveTest extends BasePacketApiLiveTest {
 
    @Test(groups = "live", dependsOnMethods = "testCreate")
    public void testReboot() {
-      api().actions(deviceId, ActionType.REBOOT);
+      api().reboot(deviceId);
       assertNodeRunning(deviceId);
    }
 
    @Test(groups = "live", dependsOnMethods = "testReboot")
    public void testPowerOff() {
-      api().actions(deviceId, ActionType.POWER_OFF);
-      assertNodeTerminated(deviceId);
+      api().powerOff(deviceId);
+      assertNodeSuspended(deviceId);
    }
 
    @Test(groups = "live", dependsOnMethods = "testPowerOff")
    public void testPowerOn() {
-      api().actions(deviceId, ActionType.POWER_ON);
+      api().powerOn(deviceId);
       assertNodeRunning(deviceId);
    }
-   
+
    @Test(dependsOnMethods = "testCreate")
    public void testList() {
       final AtomicInteger found = new AtomicInteger(0);
@@ -123,7 +122,7 @@ public class DeviceApiLiveTest extends BasePacketApiLiveTest {
       assertTrue(found.get() > 0, "Expected some devices to be returned");
    }
 
-   @Test(dependsOnMethods = "testList", alwaysRun = true)
+   @Test(dependsOnMethods = "testPowerOn", alwaysRun = true)
    public void testDelete() throws InterruptedException {
       if (deviceId != null) {
          api().delete(deviceId);

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/test/java/org/jclouds/packet/features/DeviceApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/DeviceApiMockTest.java b/packet/src/test/java/org/jclouds/packet/features/DeviceApiMockTest.java
index 705955c..f53c206 100644
--- a/packet/src/test/java/org/jclouds/packet/features/DeviceApiMockTest.java
+++ b/packet/src/test/java/org/jclouds/packet/features/DeviceApiMockTest.java
@@ -17,7 +17,6 @@
 package org.jclouds.packet.features;
 
 import org.jclouds.packet.compute.internal.BasePacketApiMockTest;
-import org.jclouds.packet.domain.ActionType;
 import org.jclouds.packet.domain.BillingCycle;
 import org.jclouds.packet.domain.Device;
 import org.jclouds.packet.domain.Device.CreateDevice;
@@ -90,7 +89,7 @@ public class DeviceApiMockTest extends BasePacketApiMockTest {
       assertEquals(device, objectFromResource("/device.json", Device.class));
 
       assertEquals(server.getRequestCount(), 1);
-      assertSent(server, "GET", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/1");
+      assertSent(server, "GET", "/devices/1");
    }
 
    public void testGetDeviceReturns404() throws InterruptedException {
@@ -101,7 +100,7 @@ public class DeviceApiMockTest extends BasePacketApiMockTest {
       assertNull(device);
 
       assertEquals(server.getRequestCount(), 1);
-      assertSent(server, "GET", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/1");
+      assertSent(server, "GET", "/devices/1");
    }
 
    public void testCreateDevice() throws InterruptedException {
@@ -133,7 +132,7 @@ public class DeviceApiMockTest extends BasePacketApiMockTest {
       api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").delete("1");
 
       assertEquals(server.getRequestCount(), 1);
-      assertSent(server, "DELETE", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/1");
+      assertSent(server, "DELETE", "/devices/1");
    }
 
    public void testDeleteDeviceReturns404() throws InterruptedException {
@@ -142,43 +141,34 @@ public class DeviceApiMockTest extends BasePacketApiMockTest {
       api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").delete("1");
 
       assertEquals(server.getRequestCount(), 1);
-      assertSent(server, "DELETE", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/1");
+      assertSent(server, "DELETE", "/devices/1");
    }
 
    public void testActionPowerOn() throws InterruptedException {
       server.enqueue(jsonResponse("/power-on.json"));
 
-      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").actions("deviceId", ActionType.POWER_ON);
+      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").powerOn("deviceId");
 
       assertEquals(server.getRequestCount(), 1);
-      assertSent(server, "POST", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/deviceId/actions");
+      assertSent(server, "POST", "/devices/deviceId/actions");
    }
 
    public void testActionPowerOff() throws InterruptedException {
       server.enqueue(jsonResponse("/power-off.json"));
 
-      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").actions("deviceId", ActionType.POWER_OFF);
+      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").powerOff("deviceId");
 
       assertEquals(server.getRequestCount(), 1);
-      assertSent(server, "POST", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/deviceId/actions");
+      assertSent(server, "POST", "/devices/deviceId/actions");
    }
 
    public void testActionReboot() throws InterruptedException {
       server.enqueue(jsonResponse("/reboot.json"));
 
-      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").actions("deviceId", ActionType.REBOOT);
+      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").reboot("deviceId");
 
       assertEquals(server.getRequestCount(), 1);
-      assertSent(server, "POST", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/deviceId/actions");
+      assertSent(server, "POST", "/devices/deviceId/actions");
    }
 
-   public void testActionRescue() throws InterruptedException {
-      server.enqueue(jsonResponse("/rescue.json"));
-
-      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").actions("deviceId", ActionType.RESCUE);
-
-      assertEquals(server.getRequestCount(), 1);
-      assertSent(server, "POST", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/deviceId/actions");
-   }
-   
 }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/test/java/org/jclouds/packet/features/FacilityApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/FacilityApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/features/FacilityApiLiveTest.java
index 95fc857..0ee90da 100644
--- a/packet/src/test/java/org/jclouds/packet/features/FacilityApiLiveTest.java
+++ b/packet/src/test/java/org/jclouds/packet/features/FacilityApiLiveTest.java
@@ -18,7 +18,7 @@ package org.jclouds.packet.features;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.jclouds.packet.compute.internal.BasePacketApiMockTest;
+import org.jclouds.packet.compute.internal.BasePacketApiLiveTest;
 import org.jclouds.packet.domain.Facility;
 import org.testng.annotations.Test;
 
@@ -30,7 +30,7 @@ import static org.testng.Assert.assertTrue;
 import static org.testng.util.Strings.isNullOrEmpty;
 
 @Test(groups = "live", testName = "FacilityApiLiveTest")
-public class FacilityApiLiveTest extends BasePacketApiMockTest {
+public class FacilityApiLiveTest extends BasePacketApiLiveTest {
 
    public void testList() {
       final AtomicInteger found = new AtomicInteger(0);

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/38f0a50d/packet/src/test/resources/rescue.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/rescue.json b/packet/src/test/resources/rescue.json
deleted file mode 100644
index 91ea877..0000000
--- a/packet/src/test/resources/rescue.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
-  "type": "rescue"
-}


[4/9] jclouds-labs git commit: initial commit for Packet.net API

Posted by na...@apache.org.
initial commit for Packet.net API

- add domain objects


Project: http://git-wip-us.apache.org/repos/asf/jclouds-labs/repo
Commit: http://git-wip-us.apache.org/repos/asf/jclouds-labs/commit/5408585d
Tree: http://git-wip-us.apache.org/repos/asf/jclouds-labs/tree/5408585d
Diff: http://git-wip-us.apache.org/repos/asf/jclouds-labs/diff/5408585d

Branch: refs/heads/2.0.x
Commit: 5408585d12b6e698a6efcebb3ebf3d0b534b1a32
Parents: 66bed4b
Author: Andrea Turli <an...@gmail.com>
Authored: Tue Jan 3 16:04:20 2017 +0100
Committer: Ignasi Barrera <na...@apache.org>
Committed: Thu Feb 2 15:23:25 2017 +0100

----------------------------------------------------------------------
 packet/README.md                                |   1 +
 packet/pom.xml                                  | 145 +++++++++++++++++++
 .../org/jclouds/packet/domain/ActionType.java   |  39 +++++
 .../org/jclouds/packet/domain/BillingCycle.java |  56 +++++++
 .../java/org/jclouds/packet/domain/Device.java  | 108 ++++++++++++++
 .../org/jclouds/packet/domain/Distribution.java |  68 +++++++++
 .../org/jclouds/packet/domain/Facility.java     |  45 ++++++
 .../java/org/jclouds/packet/domain/Href.java    |  34 +++++
 .../org/jclouds/packet/domain/IpAddress.java    |  59 ++++++++
 .../jclouds/packet/domain/OperatingSystem.java  |  45 ++++++
 .../java/org/jclouds/packet/domain/Plan.java    |  50 +++++++
 .../java/org/jclouds/packet/domain/Pricing.java |  34 +++++
 .../java/org/jclouds/packet/domain/Project.java |  66 +++++++++
 .../java/org/jclouds/packet/domain/Specs.java   |  98 +++++++++++++
 .../java/org/jclouds/packet/domain/SshKey.java  |  55 +++++++
 15 files changed, 903 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/README.md
----------------------------------------------------------------------
diff --git a/packet/README.md b/packet/README.md
new file mode 100644
index 0000000..fe38ca5
--- /dev/null
+++ b/packet/README.md
@@ -0,0 +1 @@
+# Apache jclouds Packet (packet.net) provider

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/pom.xml
----------------------------------------------------------------------
diff --git a/packet/pom.xml b/packet/pom.xml
new file mode 100644
index 0000000..8864894
--- /dev/null
+++ b/packet/pom.xml
@@ -0,0 +1,145 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    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.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.jclouds.labs</groupId>
+        <artifactId>jclouds-labs</artifactId>
+        <version>2.1.0-SNAPSHOT</version>
+    </parent>
+
+    <!-- TODO: when out of labs, switch to org.jclouds.provider -->
+    <artifactId>packet</artifactId>
+    <name>jclouds Packet compute API</name>
+    <description>jclouds components to access an implementation of Packet's Compute Service</description>
+    <packaging>bundle</packaging>
+
+    <properties>
+        <test.packet.endpoint>https://api.packet.net/</test.packet.endpoint>
+        <test.packet.identity>projectId</test.packet.identity>
+        <test.packet.credential>api key</test.packet.credential>
+        <jclouds.osgi.export>org.jclouds.packet*;version="${project.version}"</jclouds.osgi.export>
+        <jclouds.osgi.import>org.jclouds*;version="${project.version}",*</jclouds.osgi.import>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.jclouds</groupId>
+            <artifactId>jclouds-compute</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.google.auto.service</groupId>
+            <artifactId>auto-service</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.auto.value</groupId>
+            <artifactId>auto-value</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.jclouds</groupId>
+            <artifactId>jclouds-core</artifactId>
+            <version>${project.parent.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.jclouds</groupId>
+            <artifactId>jclouds-compute</artifactId>
+            <version>${project.parent.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.jclouds.driver</groupId>
+            <artifactId>jclouds-slf4j</artifactId>
+            <version>${project.parent.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.jclouds.driver</groupId>
+            <artifactId>jclouds-slf4j</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.jclouds.driver</groupId>
+            <artifactId>jclouds-sshj</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp</groupId>
+            <artifactId>mockwebserver</artifactId>
+            <scope>test</scope>
+            <exclusions>
+                <!-- Already provided by jclouds-sshj -->
+                <exclusion>
+                    <groupId>org.bouncycastle</groupId>
+                    <artifactId>bcprov-jdk15on</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+    </dependencies>
+
+    <profiles>
+        <profile>
+            <id>live</id>
+
+            <build>
+                <defaultGoal>clean verify</defaultGoal>
+
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-surefire-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>integration</id>
+                                <phase>integration-test</phase>
+                                <goals>
+                                    <goal>test</goal>
+                                </goals>
+                                <configuration>
+                                    <systemPropertyVariables>
+                                        <test.packet.endpoint>${test.packet.endpoint}</test.packet.endpoint>
+                                        <test.packet.identity>${test.packet.identity}</test.packet.identity>
+                                        <test.packet.credential>${test.packet.credential}</test.packet.credential>
+                                    </systemPropertyVariables>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
+</project>

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/ActionType.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/ActionType.java b/packet/src/main/java/org/jclouds/packet/domain/ActionType.java
new file mode 100644
index 0000000..9177ec5
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/ActionType.java
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+/**
+ * Performs an action for the given device. Possible actions include:
+
+ power_on
+ power_off
+ reboot
+ rescue: reboot the device into rescue OS.
+ */
+public enum ActionType {
+    
+    POWER_ON ("power_on"),
+    POWER_OFF ("power_off"),
+    REBOOT ("reboot"),
+    RESCUE ("rescue");
+
+    private final String type;
+
+    ActionType(String type) {
+        this.type = type;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/BillingCycle.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/BillingCycle.java b/packet/src/main/java/org/jclouds/packet/domain/BillingCycle.java
new file mode 100644
index 0000000..ef88946
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/BillingCycle.java
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.List;
+
+import com.google.common.base.Predicate;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.collect.Iterables.tryFind;
+import static java.util.Arrays.asList;
+
+public enum BillingCycle {
+   HOURLY("hourly"),
+   MONTHLY("monthly"),
+   UNRECOGNIZED("");
+
+   private static final List<BillingCycle> values = asList(BillingCycle.values());
+
+   private final String value;
+
+   private BillingCycle(String value) {
+      this.value = checkNotNull(value, "value cannot be null");
+   }
+
+   public String value() {
+      return this.value;
+   }
+
+   public static BillingCycle fromValue(String value) {
+      return tryFind(values, hasValue(value)).or(UNRECOGNIZED);
+   }
+
+   private static Predicate<BillingCycle> hasValue(final String value) {
+      return new Predicate<BillingCycle>() {
+         @Override
+         public boolean apply(BillingCycle input) {
+            return input.value.equalsIgnoreCase(value);
+         }
+      };
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/Device.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Device.java b/packet/src/main/java/org/jclouds/packet/domain/Device.java
new file mode 100644
index 0000000..af59970
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/Device.java
@@ -0,0 +1,108 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.Date;
+import java.util.List;
+
+import org.jclouds.javax.annotation.Nullable;
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.base.Enums;
+import com.google.common.base.Joiner;
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableList;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+@AutoValue
+public abstract class Device {
+
+    public enum State {
+        PROVISIONING, QUEUED, ACTIVE;
+
+        public static State fromValue(String value) {
+            Optional<State> state = Enums.getIfPresent(State.class, value.toUpperCase());
+            checkArgument(state.isPresent(), "Expected one of %s but was %s", Joiner.on(',').join(State.values()), value);
+            return state.get();
+        }
+    }
+
+    public abstract String id();
+    public abstract String shortId();
+    public abstract String hostname();
+    @Nullable
+    public abstract String description();
+    public abstract State state();
+    public abstract List<String> tags();
+    public abstract String billingCycle();
+    public abstract String user();
+    public abstract String iqn();
+    public abstract Boolean locked();
+    public abstract String bondingMode();
+    public abstract Date createdAt();
+    public abstract Date updatedAt();
+    public abstract OperatingSystem operatingSystem();
+    public abstract Facility facility();
+    public abstract Href project();
+    public abstract Href projectLite();
+    public abstract List<Object> volumes();
+    public abstract List<IpAddress> ipAddresses();
+    public abstract Plan plan();
+    public abstract String rootPassword();
+    public abstract String userdata();
+    public abstract String href();
+
+    @SerializedNames({"id", "short_id", "hostname", "description", "state", "tags", "billing_cycle", "user", "iqn", "locked", "bonding_mode", "created_at", "updated_at", "operating_system", "facility", "project", "project_lite", "volumes", "ip_addresses", "plan", "root_password", "userdata", "href"})
+    public static Device create(String id,
+                                String shortId,
+                                String hostname,
+                                String description,
+                                State state,
+                                List<String> tags,
+                                String billingCycle,
+                                String user,
+                                String iqn,
+                                Boolean locked,
+                                String bondingMode,
+                                Date createdAt,
+                                Date updatedAt,
+                                OperatingSystem operatingSystem,
+                                Facility facility,
+                                Href project,
+                                Href projectLite,
+                                List<Object> volumes,
+                                List<IpAddress> ipAddresses,
+                                Plan plan,
+                                String rootPassword,
+                                String userdata,
+                                String href
+    ) {
+        return new AutoValue_Device(id, shortId, hostname, description, state,
+                tags == null ? ImmutableList.<String> of() : ImmutableList.copyOf(tags),
+                billingCycle, user, iqn, locked, bondingMode, createdAt, updatedAt, operatingSystem, facility, project, projectLite,
+                volumes == null ? ImmutableList.of() : ImmutableList.copyOf(volumes),
+                ipAddresses == null ? ImmutableList.<IpAddress>of() : ImmutableList.copyOf(ipAddresses),
+                plan, rootPassword, userdata, href
+        );
+    }
+
+    Device() {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/Distribution.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Distribution.java b/packet/src/main/java/org/jclouds/packet/domain/Distribution.java
new file mode 100644
index 0000000..0692d59
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/Distribution.java
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.List;
+
+import org.jclouds.compute.domain.OsFamily;
+
+import com.google.common.base.Predicate;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.collect.Iterables.tryFind;
+import static java.util.Arrays.asList;
+
+public enum Distribution {
+   
+   CENTOS(OsFamily.CENTOS, "centos"),
+   COREOS(OsFamily.COREOS, "coreos"),
+   DEBIAN(OsFamily.DEBIAN, "debian"),
+   UBUNTU(OsFamily.UBUNTU, "ubuntu"),
+   WINDOWS(OsFamily.WINDOWS, "windows"),
+   UNRECOGNIZED(OsFamily.UNRECOGNIZED, "");
+
+   private static final List<Distribution> values = asList(Distribution.values());
+
+   private final OsFamily osFamily;
+   private final String value;
+
+   private Distribution(OsFamily osFamily, String value) {
+      this.osFamily = checkNotNull(osFamily, "osFamily cannot be null");
+      this.value = checkNotNull(value, "value cannot be null");
+   }
+
+   public OsFamily osFamily() {
+      return this.osFamily;
+   }
+
+   public String value() {
+      return this.value;
+   }
+
+   public static Distribution fromValue(String value) {
+      return tryFind(values, hasValue(value)).or(UNRECOGNIZED);
+   }
+
+   private static Predicate<Distribution> hasValue(final String value) {
+      return new Predicate<Distribution>() {
+         @Override
+         public boolean apply(Distribution input) {
+            return input.value.equalsIgnoreCase(value);
+         }
+      };
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/Facility.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Facility.java b/packet/src/main/java/org/jclouds/packet/domain/Facility.java
new file mode 100644
index 0000000..3124c7e
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/Facility.java
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.List;
+
+import org.jclouds.javax.annotation.Nullable;
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
+
+@AutoValue
+public abstract class Facility {
+
+    public abstract String id();
+    public abstract String name();
+    public abstract String code();
+    public abstract List<String> features();
+    @Nullable
+    public abstract String address();
+
+    @SerializedNames({"id", "name", "code", "features", "address"})
+    public static Facility create(final String id, String name, String code, List<String> features, String address) {
+        return new AutoValue_Facility(id, name, code,
+                features == null ? ImmutableList.<String> of() : ImmutableList.copyOf(features),
+                address);
+    }
+
+    Facility() {}
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/Href.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Href.java b/packet/src/main/java/org/jclouds/packet/domain/Href.java
new file mode 100644
index 0000000..493d000
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/Href.java
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+
+@AutoValue
+public abstract class Href {
+
+    public abstract String href();
+
+    @SerializedNames({ "href" })
+    public static Href create(String href) {
+        return new AutoValue_Href(href);
+    }
+
+    Href() {}
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/IpAddress.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/IpAddress.java b/packet/src/main/java/org/jclouds/packet/domain/IpAddress.java
new file mode 100644
index 0000000..d8b3e1e
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/IpAddress.java
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+
+@AutoValue
+public abstract class IpAddress {
+
+    public abstract String id();
+    public abstract Integer addressFamily();
+    public abstract String netmask();
+    public abstract Boolean publicAddress();
+    public abstract Integer cidr();
+    public abstract Boolean management();
+    public abstract Boolean manageable();
+    public abstract Href assignedTo();
+    public abstract String network();
+    public abstract String address();
+    public abstract String gateway();
+    public abstract String href();
+
+    @SerializedNames({"id", "address_family", "netmask", "public", "cidr", "management", "manageable", "assigned_to", "network", "address", "gateway", "href"})
+    public static IpAddress create(
+            String id,
+            Integer addressFamily,
+            String netmask,
+            Boolean publicAddress,
+            Integer cidr,
+            Boolean management,
+            Boolean manageable,
+            Href assignedTo,
+            String network,
+            String address,
+            String gateway,
+            String href
+    ) {
+        return new AutoValue_IpAddress(id, addressFamily, netmask, publicAddress, cidr, management, manageable, assignedTo, network, address, gateway, href
+        );
+    }
+
+    IpAddress() {}
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/OperatingSystem.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/OperatingSystem.java b/packet/src/main/java/org/jclouds/packet/domain/OperatingSystem.java
new file mode 100644
index 0000000..e491661
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/OperatingSystem.java
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.Set;
+
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableSet;
+
+@AutoValue
+public abstract class OperatingSystem {
+
+    public abstract String id();
+    public abstract String slug();
+    public abstract String name();
+    public abstract String distribution();
+    public abstract String version();
+    public abstract Set<String> provisionableOn();
+
+    @SerializedNames({"id", "slug", "name", "distro", "version", "provisionable_on"})
+    public static OperatingSystem create(String id, String slug, String name, String distribution, String version, Set<String> provisionableOn) {
+        return new AutoValue_OperatingSystem(id, slug, name, distribution, version,
+                provisionableOn == null ? ImmutableSet.<String> of() : ImmutableSet.copyOf(provisionableOn)
+        );
+    }
+
+    OperatingSystem() {}
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/Plan.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Plan.java b/packet/src/main/java/org/jclouds/packet/domain/Plan.java
new file mode 100644
index 0000000..95683b6
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/Plan.java
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.List;
+
+import org.jclouds.javax.annotation.Nullable;
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
+
+@AutoValue
+public abstract class Plan {
+
+    public abstract String id();
+    public abstract String slug();
+    public abstract String name();
+    public abstract String description();
+    public abstract String line();
+    @Nullable
+    public abstract Specs specs();
+    public abstract List<Href> availableIn();
+    public abstract Pricing pricing();
+
+    @SerializedNames({"id", "slug", "name", "description", "line", "specs", "available_in", "pricing"})
+    public static Plan create(final String id, String slug, String name, String description, String line, Specs specs, List<Href> availableIn, Pricing pricing) {
+        return new AutoValue_Plan(id, slug, name, description, line,
+                specs,
+                availableIn == null ? ImmutableList.<Href> of() : ImmutableList.copyOf(availableIn),
+                pricing
+        );
+    }
+
+    Plan() {}
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/Pricing.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Pricing.java b/packet/src/main/java/org/jclouds/packet/domain/Pricing.java
new file mode 100644
index 0000000..9ca7435
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/Pricing.java
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+
+@AutoValue
+public abstract class Pricing {
+
+    public abstract double hour();
+
+    @SerializedNames({ "hour" })
+    public static Pricing create(double hour) {
+        return new AutoValue_Pricing(hour);
+    }
+
+    Pricing() {}
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/Project.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Project.java b/packet/src/main/java/org/jclouds/packet/domain/Project.java
new file mode 100644
index 0000000..69fe077
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/Project.java
@@ -0,0 +1,66 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+@AutoValue
+public abstract class Project {
+
+    public abstract String id();
+    public abstract String name();
+    public abstract Date createdAt();
+    public abstract Date updatedAt();
+    public abstract Map<String, Object> maxDevices();
+    public abstract List<Href> members();
+    public abstract List<Href> memberships();
+    public abstract List<Href> invitations();
+    public abstract Href paymentMethod();
+    public abstract List<Href> devices();
+    public abstract List<Href> sshKeys();
+    public abstract List<Href> volumes();
+    public abstract String href();
+
+    @SerializedNames({"id", "name", "created_at", "updated_at", "max_devices", "members", "memberships", "invitations", "payment_method", "devices", "ssh_keys", "volumes", "href"})
+    public static Project create(String id, String name, Date createdAt, Date updatedAt, Map<String, Object> maxDevices,
+                                 List<Href> members, List<Href> memberships, List<Href> invitations, Href paymentMethod,
+                                 List<Href> devices,
+                                 List<Href> sshKeys,
+                                 List<Href> volumes,
+                                 String href) {
+        return new AutoValue_Project(id, name, createdAt, updatedAt,
+                maxDevices == null ? ImmutableMap.<String, Object> of() : ImmutableMap.copyOf(maxDevices),
+                members == null ? ImmutableList.<Href> of() : ImmutableList.copyOf(members),
+                memberships == null ? ImmutableList.<Href> of() : ImmutableList.copyOf(memberships),
+                invitations == null ? ImmutableList.<Href> of() : ImmutableList.copyOf(invitations),
+                paymentMethod,
+                devices == null ? ImmutableList.<Href> of() : ImmutableList.copyOf(devices),
+                sshKeys == null ? ImmutableList.<Href> of() : ImmutableList.copyOf(sshKeys),
+                volumes == null ? ImmutableList.<Href> of() : ImmutableList.copyOf(volumes),
+                href);
+    }
+
+    Project() {}
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/Specs.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Specs.java b/packet/src/main/java/org/jclouds/packet/domain/Specs.java
new file mode 100644
index 0000000..bafd344
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/Specs.java
@@ -0,0 +1,98 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.List;
+import java.util.Map;
+
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+@AutoValue
+public abstract class Specs {
+
+    @AutoValue
+    public abstract static class NIC {
+
+        public abstract Integer count();
+        public abstract String type();
+
+        @SerializedNames({ "count", "type" })
+        public static NIC create(Integer count, String type) {
+            return new AutoValue_Specs_NIC(count, type);
+        }
+    }
+
+    @AutoValue
+    public abstract static class Drive {
+
+        public abstract Integer count();
+        public abstract String size();
+        public abstract String type();
+
+        @SerializedNames({ "count", "size", "type" })
+        public static Drive create(Integer count, String size, String type) {
+            return new AutoValue_Specs_Drive(count, size, type);
+        }
+    }
+
+    @AutoValue
+    public abstract static class CPU {
+
+        public abstract Integer count();
+        public abstract String type();
+
+        @SerializedNames({ "count", "type" })
+        public static CPU create(Integer count, String type) {
+            return new AutoValue_Specs_CPU(count, type);
+        }
+    }
+
+    @AutoValue
+    public abstract static class Memory {
+
+        public abstract String total();
+
+        @SerializedNames({ "total" })
+        public static Memory create(String total) {
+            return new AutoValue_Specs_Memory(total);
+        }
+    }
+
+    public abstract List<CPU> cpus();
+    public abstract Memory memory();
+    public abstract List<Drive> drives();
+    public abstract List<NIC> nics();
+    public abstract Map<String, Object> features();
+
+    @SerializedNames({"cpus", "memory", "drives", "nics", "features"})
+    public static Specs create(List<CPU> cpus, Memory memory, List<Drive> drives, List<NIC> nics, Map<String, Object> features) {
+        return new AutoValue_Specs(
+                cpus == null ? ImmutableList.<CPU> of() : ImmutableList.copyOf(cpus),
+                memory,
+                drives == null ? ImmutableList.<Drive> of() : ImmutableList.copyOf(drives),
+                nics == null ? ImmutableList.<NIC> of() : ImmutableList.copyOf(nics),
+                features == null ? ImmutableMap.<String, Object> of() : ImmutableMap.copyOf(features)
+        );
+    }
+
+    Specs() {}
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5408585d/packet/src/main/java/org/jclouds/packet/domain/SshKey.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/SshKey.java b/packet/src/main/java/org/jclouds/packet/domain/SshKey.java
new file mode 100644
index 0000000..82e9c00
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/SshKey.java
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.Date;
+
+import org.jclouds.javax.annotation.Nullable;
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+
+@AutoValue
+public abstract class SshKey {
+
+    @AutoValue
+    public abstract static class Owner {
+
+        public abstract String href();
+
+        @SerializedNames({ "href" })
+        public static Owner create(String href) {
+            return new AutoValue_SshKey_Owner(href);
+        }
+    }
+
+    public abstract String id();
+    public abstract String label();
+    public abstract String key();
+    public abstract String fingerprint();
+    public abstract Date createdAt();
+    public abstract Date updatedAt();
+    @Nullable public abstract Owner owner();
+    public abstract String href();
+
+    @SerializedNames({"id", "label", "key", "fingerprint", "created_at", "updated_at", "owner", "href"})
+    public static SshKey create(String id, String label, String key, String fingerprint, Date createdAt, Date updatedAt, Owner owner, String href) {
+        return new AutoValue_SshKey(id, label, key, fingerprint, createdAt, updatedAt, owner, href);
+    }
+
+    SshKey() {}
+}


[3/9] jclouds-labs git commit: add ApiMetadata and ProviderMetadata

Posted by na...@apache.org.
add ApiMetadata and ProviderMetadata

- add skeleton for PacketApi with ProjectApi only
- add XAuthTokenToRequest filter
- add HttpApiModule and ParserModule
- add ProjectApi feature with Mock and Live Tests
- add pagination to Project API
- add test pagination


Project: http://git-wip-us.apache.org/repos/asf/jclouds-labs/repo
Commit: http://git-wip-us.apache.org/repos/asf/jclouds-labs/commit/b7f39234
Tree: http://git-wip-us.apache.org/repos/asf/jclouds-labs/tree/b7f39234
Diff: http://git-wip-us.apache.org/repos/asf/jclouds-labs/diff/b7f39234

Branch: refs/heads/2.0.x
Commit: b7f392343c3cfb416b5e33d14c9ca3bd9fa5d519
Parents: 5408585
Author: Andrea Turli <an...@gmail.com>
Authored: Tue Jan 10 11:11:18 2017 +0100
Committer: Ignasi Barrera <na...@apache.org>
Committed: Thu Feb 2 15:23:25 2017 +0100

----------------------------------------------------------------------
 .../main/java/org/jclouds/packet/PacketApi.java |  40 +++
 .../org/jclouds/packet/PacketApiMetadata.java   |  86 +++++
 .../jclouds/packet/PacketProviderMetadata.java  |  78 +++++
 .../config/PacketComputeParserModule.java       |  30 ++
 .../packet/config/PacketHttpApiModule.java      |  55 ++++
 .../domain/internal/PaginatedCollection.java    |  84 +++++
 .../packet/domain/options/ListOptions.java      |  60 ++++
 .../org/jclouds/packet/features/ProjectApi.java |  93 ++++++
 .../packet/filters/AddXAuthTokenToRequest.java  |  47 +++
 .../packet/functions/BaseToPagedIterable.java   |  59 ++++
 .../packet/functions/LinkToListOptions.java     |  63 ++++
 .../packet/handlers/PacketErrorHandler.java     |  64 ++++
 .../packet/PacketProviderMetadataTest.java      |  29 ++
 .../compute/internal/BasePacketApiLiveTest.java |  73 +++++
 .../compute/internal/BasePacketApiMockTest.java | 145 +++++++++
 .../packet/features/ProjectApiLiveTest.java     |  63 ++++
 .../packet/features/ProjectApiMockTest.java     |  79 +++++
 .../packet/functions/LinkToListOptionsTest.java |  57 ++++
 packet/src/test/resources/logback-test.xml      |  42 +++
 packet/src/test/resources/projects-first.json   | 315 +++++++++++++++++++
 packet/src/test/resources/projects-last.json    | 197 ++++++++++++
 packet/src/test/resources/projects.json         |   1 +
 22 files changed, 1760 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/PacketApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/PacketApi.java b/packet/src/main/java/org/jclouds/packet/PacketApi.java
new file mode 100644
index 0000000..1cb8e9b
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/PacketApi.java
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet;
+
+import java.io.Closeable;
+
+import org.jclouds.packet.features.ProjectApi;
+import org.jclouds.rest.annotations.Delegate;
+
+/**
+ * The Packet API is a REST API for managing your services and deployments.
+ * <p>
+ *
+ * @see <a href="https://www.packet.net/help/api/" >doc</a>
+ */
+public interface PacketApi extends Closeable {
+
+   /**
+    * The Packet API includes operations for managing project.
+    *
+    * @see <a href="https://www.packet.net/help/api/#page:projects,header:projects-projects">docs</a>
+    */
+   @Delegate
+   ProjectApi projectApi();
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java b/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
new file mode 100644
index 0000000..75ba0e6
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
@@ -0,0 +1,86 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet;
+
+import java.net.URI;
+import java.util.Properties;
+
+import org.jclouds.apis.ApiMetadata;
+import org.jclouds.packet.config.PacketComputeParserModule;
+import org.jclouds.packet.config.PacketHttpApiModule;
+import org.jclouds.rest.internal.BaseHttpApiMetadata;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.inject.Module;
+
+import static org.jclouds.compute.config.ComputeServiceProperties.TEMPLATE;
+import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_RUNNING;
+
+/**
+ * Implementation of {@link ApiMetadata} for Packet API
+ */
+public class PacketApiMetadata extends BaseHttpApiMetadata<PacketApi> {
+
+   @Override
+   public Builder toBuilder() {
+      return new Builder().fromApiMetadata(this);
+   }
+
+   public PacketApiMetadata() {
+      this(new Builder());
+   }
+
+   protected PacketApiMetadata(Builder builder) {
+      super(builder);
+   }
+
+   public static Properties defaultProperties() {
+      Properties properties = BaseHttpApiMetadata.defaultProperties();
+      properties.put(TEMPLATE, "osFamily=UBUNTU,os64Bit=true,osVersionMatches=16.*");
+      properties.put(TIMEOUT_NODE_RUNNING, 300000); // 5 mins
+      return properties;
+   }
+
+   public static class Builder extends BaseHttpApiMetadata.Builder<PacketApi, Builder> {
+
+      protected Builder() {
+         id("packet")
+                 .name("Packet API")
+                 .identityName("Packet Project Id")
+                 .credentialName("Must be Packet Token")
+                 .documentation(URI.create("https://www.packet.net/help/api/#"))
+                 .defaultEndpoint("https://api.packet.net")
+                 .defaultProperties(PacketApiMetadata.defaultProperties())
+                 //.view(typeToken(ComputeServiceContext.class))
+                 .defaultModules(ImmutableSet.<Class<? extends Module>>builder()
+                         .add(PacketHttpApiModule.class)
+                         .add(PacketComputeParserModule.class)
+                         //.add(PacketComputeServiceContextModule.class)
+                         .build());
+      }
+
+      @Override
+      public PacketApiMetadata build() {
+         return new PacketApiMetadata(this);
+      }
+
+      @Override
+      protected Builder self() {
+         return this;
+      }
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/PacketProviderMetadata.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/PacketProviderMetadata.java b/packet/src/main/java/org/jclouds/packet/PacketProviderMetadata.java
new file mode 100644
index 0000000..69e0918
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/PacketProviderMetadata.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet;
+
+import java.net.URI;
+import java.util.Properties;
+
+import org.jclouds.providers.ProviderMetadata;
+import org.jclouds.providers.internal.BaseProviderMetadata;
+
+import com.google.auto.service.AutoService;
+
+@AutoService(ProviderMetadata.class)
+public class PacketProviderMetadata extends BaseProviderMetadata {
+
+    public static Builder builder() {
+        return new Builder();
+    }
+
+    @Override
+    public Builder toBuilder() {
+        return builder().fromProviderMetadata(this);
+    }
+
+    public PacketProviderMetadata() {
+        super(builder());
+    }
+
+    public PacketProviderMetadata(Builder builder) {
+        super(builder);
+    }
+
+    public static Properties defaultProperties() {
+        final Properties properties = PacketApiMetadata.defaultProperties();
+        return properties;
+    }
+
+    public static class Builder extends BaseProviderMetadata.Builder {
+
+        protected Builder() {
+            id("packet")
+                    .name("Packet Compute Services")
+                    .apiMetadata(new PacketApiMetadata())
+                    .homepage(URI.create("https://www.packet.net/"))
+                    .console(URI.create("https://app.packet.net/portal"))
+                    .endpoint("https://api.packet.net")
+                    .iso3166Codes("US-CA", "US-NJ", "NL", "JP")
+                    .defaultProperties(PacketProviderMetadata.defaultProperties());
+        }
+
+        @Override
+        public PacketProviderMetadata build() {
+            return new PacketProviderMetadata(this);
+        }
+
+        @Override
+        public Builder fromProviderMetadata(ProviderMetadata in) {
+            super.fromProviderMetadata(in);
+            return this;
+        }
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/config/PacketComputeParserModule.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/config/PacketComputeParserModule.java b/packet/src/main/java/org/jclouds/packet/config/PacketComputeParserModule.java
new file mode 100644
index 0000000..8471fc4
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/config/PacketComputeParserModule.java
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.config;
+
+import org.jclouds.json.config.GsonModule;
+
+import com.google.inject.AbstractModule;
+
+public class PacketComputeParserModule extends AbstractModule {
+
+   @Override
+   protected void configure() {
+      bind(GsonModule.DateAdapter.class).to(GsonModule.Iso8601DateAdapter.class);
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/config/PacketHttpApiModule.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/config/PacketHttpApiModule.java b/packet/src/main/java/org/jclouds/packet/config/PacketHttpApiModule.java
new file mode 100644
index 0000000..e74bb19
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/config/PacketHttpApiModule.java
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.config;
+
+import org.jclouds.http.HttpErrorHandler;
+import org.jclouds.http.annotation.ClientError;
+import org.jclouds.http.annotation.Redirection;
+import org.jclouds.http.annotation.ServerError;
+import org.jclouds.location.suppliers.ImplicitLocationSupplier;
+import org.jclouds.location.suppliers.implicit.FirstRegion;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.options.ListOptions;
+import org.jclouds.packet.functions.LinkToListOptions;
+import org.jclouds.packet.handlers.PacketErrorHandler;
+import org.jclouds.rest.ConfiguresHttpApi;
+import org.jclouds.rest.config.HttpApiModule;
+
+import com.google.common.base.Function;
+import com.google.inject.Scopes;
+import com.google.inject.TypeLiteral;
+
+@ConfiguresHttpApi
+public class PacketHttpApiModule extends HttpApiModule<PacketApi> {
+
+   @Override
+   protected void configure() {
+      super.configure();
+      bind(ImplicitLocationSupplier.class).to(FirstRegion.class).in(Scopes.SINGLETON);
+      bind(new TypeLiteral<Function<Href, ListOptions>>() {
+      }).to(LinkToListOptions.class);
+   }
+
+   @Override
+   protected void bindErrorHandlers() {
+      bind(HttpErrorHandler.class).annotatedWith(Redirection.class).to(PacketErrorHandler.class);
+      bind(HttpErrorHandler.class).annotatedWith(ClientError.class).to(PacketErrorHandler.class);
+      bind(HttpErrorHandler.class).annotatedWith(ServerError.class).to(PacketErrorHandler.class);
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/domain/internal/PaginatedCollection.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/internal/PaginatedCollection.java b/packet/src/main/java/org/jclouds/packet/domain/internal/PaginatedCollection.java
new file mode 100644
index 0000000..047151e
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/internal/PaginatedCollection.java
@@ -0,0 +1,84 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain.internal;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.javax.annotation.Nullable;
+import org.jclouds.json.SerializedNames;
+import org.jclouds.packet.domain.Href;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Base class for all collections that return paginated results.
+ */
+public abstract class PaginatedCollection<T> extends IterableWithMarker<T> {
+
+   @AutoValue
+   public abstract static class Meta {
+      public abstract long total();
+      @Nullable public abstract Href first();
+      @Nullable public abstract Href previous();
+      @Nullable public abstract Href self();
+      @Nullable public abstract Href next();
+      @Nullable public abstract Href last();
+
+      @SerializedNames({ "total", "first", "previous", "self", "next", "last" })
+      public static Meta create(long total, Href first, Href previous, Href self, Href next, Href last) {
+         return new AutoValue_PaginatedCollection_Meta(total, first, previous, self, next, last);
+      }
+
+      Meta() { }
+   }
+
+   private final List<T> items;
+   private final Meta meta;
+
+   protected PaginatedCollection(List<T> items, Meta meta) {
+      this.items = ImmutableList.copyOf(checkNotNull(items, "items cannot be null"));
+      this.meta = checkNotNull(meta, "meta cannot be null");
+   }
+
+   public List<T> items() {
+      return items;
+   }
+
+   public Meta meta() {
+      return meta;
+   }
+
+   @Override
+   public Iterator<T> iterator() {
+      return items.iterator();
+   }
+
+   @Override
+   public Optional<Object> nextMarker() {
+      if (meta.next() == null) {
+         return Optional.absent();
+      }
+      return Optional.fromNullable((Object) meta.next());
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/domain/options/ListOptions.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/options/ListOptions.java b/packet/src/main/java/org/jclouds/packet/domain/options/ListOptions.java
new file mode 100644
index 0000000..c858a7f
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/options/ListOptions.java
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain.options;
+
+import org.jclouds.http.options.BaseHttpRequestOptions;
+
+/**
+ * Options to customize how paginated lists are returned.
+ */
+public class ListOptions extends BaseHttpRequestOptions {
+   public static final String PAGE_PARAM = "page";
+   public static final String PER_PAGE_PARAM = "per_page";
+   
+   /**
+    * Configures the number of entries to return in each page.
+    */
+   public ListOptions perPage(int perPage) {
+      queryParameters.put(PER_PAGE_PARAM, String.valueOf(perPage));
+      return this;
+   }
+   
+   /**
+    * Configures the number of the page to be returned.
+    */
+   public ListOptions page(int page) {
+      queryParameters.put(PAGE_PARAM, String.valueOf(page));
+      return this;
+   }
+   
+   public static final class Builder {
+      
+      /**
+       * @see {@link ListOptions#perPage(int)}
+       */
+      public static ListOptions perPage(int perPage) {
+         return new ListOptions().perPage(perPage);
+      }
+      
+      /**
+       * @see {@link ListOptions#page(int)}
+       */
+      public static ListOptions page(int page) {
+         return new ListOptions().page(page);
+      }
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java b/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
new file mode 100644
index 0000000..e6bf0ca
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
@@ -0,0 +1,93 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.beans.ConstructorProperties;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.MediaType;
+
+import org.jclouds.Fallbacks;
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.collect.PagedIterable;
+import org.jclouds.http.functions.ParseJson;
+import org.jclouds.json.Json;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.Project;
+import org.jclouds.packet.domain.internal.PaginatedCollection;
+import org.jclouds.packet.domain.options.ListOptions;
+import org.jclouds.packet.filters.AddXAuthTokenToRequest;
+import org.jclouds.packet.functions.BaseToPagedIterable;
+import org.jclouds.rest.annotations.Fallback;
+import org.jclouds.rest.annotations.RequestFilters;
+import org.jclouds.rest.annotations.ResponseParser;
+import org.jclouds.rest.annotations.Transform;
+
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+import com.google.inject.TypeLiteral;
+
+@Path("/projects")
+@Consumes(MediaType.APPLICATION_JSON)
+@RequestFilters(AddXAuthTokenToRequest.class)
+public interface ProjectApi {
+
+
+    @Named("project:list")
+    @GET
+    @ResponseParser(ParseProjects.class)
+    @Transform(ParseProjects.ToPagedIterable.class)
+    @Fallback(Fallbacks.EmptyPagedIterableOnNotFoundOr404.class)
+    PagedIterable<Project> list();
+
+    @Named("project:list")
+    @GET
+    @ResponseParser(ParseProjects.class)
+    @Fallback(Fallbacks.EmptyIterableWithMarkerOnNotFoundOr404.class)
+    IterableWithMarker<Project> list(ListOptions options);
+
+    final class ParseProjects extends ParseJson<ParseProjects.Projects> {
+        @Inject
+        ParseProjects(Json json) {
+            super(json, TypeLiteral.get(Projects.class));
+        }
+
+        private static class Projects extends PaginatedCollection<Project> {
+            @ConstructorProperties({ "projects", "meta" })
+            public Projects(List<Project> items, Meta meta) {
+                super(items, meta);
+            }
+        }
+
+        private static class ToPagedIterable extends BaseToPagedIterable<Project, ListOptions> {
+            @Inject ToPagedIterable(PacketApi api, Function<Href, ListOptions> linkToOptions) {
+                super(api, linkToOptions);
+            }
+
+            @Override
+            protected IterableWithMarker<Project> fetchPageUsingOptions(ListOptions options, Optional<Object> arg0) {
+                return api.projectApi().list(options);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/filters/AddXAuthTokenToRequest.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/filters/AddXAuthTokenToRequest.java b/packet/src/main/java/org/jclouds/packet/filters/AddXAuthTokenToRequest.java
new file mode 100644
index 0000000..e9d6bdd
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/filters/AddXAuthTokenToRequest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.filters;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+
+import org.jclouds.domain.Credentials;
+import org.jclouds.http.HttpException;
+import org.jclouds.http.HttpRequest;
+import org.jclouds.http.HttpRequestFilter;
+import org.jclouds.location.Provider;
+
+import com.google.common.base.Supplier;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+@Singleton
+public class AddXAuthTokenToRequest implements HttpRequestFilter {
+
+    private final Supplier<Credentials> creds;
+
+    @Inject
+    AddXAuthTokenToRequest(@Provider Supplier<Credentials> creds) {
+        this.creds = creds;
+    }
+
+    @Override
+    public HttpRequest filter(HttpRequest request) throws HttpException {
+        Credentials currentCreds = checkNotNull(creds.get(), "credential supplier returned null");
+        return request.toBuilder().replaceHeader("X-Auth-Token", currentCreds.credential).build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/functions/BaseToPagedIterable.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/functions/BaseToPagedIterable.java b/packet/src/main/java/org/jclouds/packet/functions/BaseToPagedIterable.java
new file mode 100644
index 0000000..c5c275b
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/functions/BaseToPagedIterable.java
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.functions;
+
+import javax.inject.Inject;
+
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.collect.internal.Arg0ToPagedIterable;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.options.ListOptions;
+
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+
+/**
+ * Base class to implement the functions that build the
+ * <code>PagedIterable</code>. Subclasses just need to override the
+ * {@link #fetchPageUsingOptions(ListOptions, Optional)} to invoke the right API
+ * method with the given options parameter to get the next page.
+ */
+public abstract class BaseToPagedIterable<T, O extends ListOptions> extends
+        Arg0ToPagedIterable<T, BaseToPagedIterable<T, O>> {
+   private final Function<Href, O> linkToOptions;
+   protected final PacketApi api;
+
+   @Inject protected BaseToPagedIterable(PacketApi api, Function<Href, O> linkToOptions) {
+      this.api = api;
+      this.linkToOptions = linkToOptions;
+   }
+
+   protected abstract IterableWithMarker<T> fetchPageUsingOptions(O options, Optional<Object> arg0);
+
+   @Override
+   protected Function<Object, IterableWithMarker<T>> markerToNextForArg0(final Optional<Object> arg0) {
+      return new Function<Object, IterableWithMarker<T>>() {
+         @Override
+         public IterableWithMarker<T> apply(Object input) {
+            O nextOptions = linkToOptions.apply(Href.class.cast(input));
+            return fetchPageUsingOptions(nextOptions, arg0);
+         }
+      };
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/functions/LinkToListOptions.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/functions/LinkToListOptions.java b/packet/src/main/java/org/jclouds/packet/functions/LinkToListOptions.java
new file mode 100644
index 0000000..4aef811
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/functions/LinkToListOptions.java
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.functions;
+
+import java.net.URI;
+
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.options.ListOptions;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Multimap;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Strings.emptyToNull;
+import static com.google.common.collect.Iterables.getFirst;
+import static org.jclouds.http.utils.Queries.queryParser;
+import static org.jclouds.packet.domain.options.ListOptions.PAGE_PARAM;
+import static org.jclouds.packet.domain.options.ListOptions.PER_PAGE_PARAM;
+
+/**
+ * Transforms an href returned by the API into a {@link ListOptions} that can be
+ * used to perform a request to get another page of a paginated list.
+ */
+public class LinkToListOptions implements Function<Href, ListOptions> {
+
+   @Override
+   public ListOptions apply(Href input) {
+      checkNotNull(input, "input cannot be null");
+
+      Multimap<String, String> queryParams = queryParser().apply(URI.create(input.href()).getQuery());
+      String nextPage = getFirstOrNull(PAGE_PARAM, queryParams);
+      String nextPerPage = getFirstOrNull(PER_PAGE_PARAM, queryParams);
+
+      ListOptions options = new ListOptions();
+      if (nextPage != null) {
+         options.page(Integer.parseInt(nextPage));
+      }
+      if (nextPerPage != null) {
+         options.perPage(Integer.parseInt(nextPerPage));
+      }
+
+      return options;
+   }
+
+   public static String getFirstOrNull(String key, Multimap<String, String> params) {
+      return params.containsKey(key) ? emptyToNull(getFirst(params.get(key), null)) : null;
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/main/java/org/jclouds/packet/handlers/PacketErrorHandler.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/handlers/PacketErrorHandler.java b/packet/src/main/java/org/jclouds/packet/handlers/PacketErrorHandler.java
new file mode 100644
index 0000000..01d870b
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/handlers/PacketErrorHandler.java
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.handlers;
+
+import javax.inject.Singleton;
+
+import org.jclouds.http.HttpCommand;
+import org.jclouds.http.HttpErrorHandler;
+import org.jclouds.http.HttpResponse;
+import org.jclouds.http.HttpResponseException;
+import org.jclouds.rest.AuthorizationException;
+import org.jclouds.rest.ResourceNotFoundException;
+
+import static org.jclouds.http.HttpUtils.closeClientButKeepContentStream;
+
+/**
+ * This will parse and set an appropriate exception on the command object.
+ */
+@Singleton
+public class PacketErrorHandler implements HttpErrorHandler {
+
+   public void handleError(HttpCommand command, HttpResponse response) {
+      // it is important to always read fully and close streams
+      byte[] data = closeClientButKeepContentStream(response);
+      String message = data != null ? new String(data) : null;
+
+      Exception exception = message != null ? new HttpResponseException(command, response, message)
+              : new HttpResponseException(command, response);
+      message = message != null ? message : String.format("%s -> %s", command.getCurrentRequest().getRequestLine(),
+              response.getStatusLine());
+      switch (response.getStatusCode()) {
+         case 400:
+            exception = new IllegalArgumentException(message, exception);
+            break;
+         case 401:
+         case 403:
+               exception = new AuthorizationException(message, exception);
+            break;
+         case 404:
+            if (!command.getCurrentRequest().getMethod().equals("DELETE")) {
+               exception = new ResourceNotFoundException(message, exception);
+            }
+            break;
+         case 409:
+            exception = new IllegalStateException(message, exception);
+            break;
+      }
+      command.setException(exception);
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/java/org/jclouds/packet/PacketProviderMetadataTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/PacketProviderMetadataTest.java b/packet/src/test/java/org/jclouds/packet/PacketProviderMetadataTest.java
new file mode 100644
index 0000000..cc0c8c5
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/PacketProviderMetadataTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet;
+
+import org.jclouds.providers.internal.BaseProviderMetadataTest;
+import org.testng.annotations.Test;
+
+@Test(groups = "unit", testName = "PacketProviderMetadataTest")
+public class PacketProviderMetadataTest extends BaseProviderMetadataTest {
+
+   public PacketProviderMetadataTest() {
+      super(new PacketProviderMetadata(), new PacketApiMetadata());
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiLiveTest.java
new file mode 100644
index 0000000..6c8cf63
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiLiveTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.internal;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.jclouds.apis.BaseApiLiveTest;
+import org.jclouds.compute.config.ComputeServiceProperties;
+import org.jclouds.packet.PacketApi;
+
+import com.google.common.base.Predicate;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import com.google.inject.Module;
+import com.google.inject.TypeLiteral;
+import com.google.inject.name.Names;
+
+import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_RUNNING;
+import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_TERMINATED;
+import static org.testng.Assert.assertTrue;
+
+public class BasePacketApiLiveTest extends BaseApiLiveTest<PacketApi> {
+
+   private Predicate<String> deviceRunning;
+   private Predicate<String> deviceTerminated;
+
+   public BasePacketApiLiveTest() {
+      provider = "packet";
+   }
+
+   @Override
+   protected Properties setupProperties() {
+      Properties props = super.setupProperties();
+      props.put(ComputeServiceProperties.POLL_INITIAL_PERIOD, 1000);
+      props.put(ComputeServiceProperties.POLL_MAX_PERIOD, 10000);
+      props.put(ComputeServiceProperties.TIMEOUT_IMAGE_AVAILABLE, TimeUnit.MINUTES.toMillis(45));
+      return props;
+   }
+
+   @Override
+   protected PacketApi create(Properties props, Iterable<Module> modules) {
+      Injector injector = newBuilder().modules(modules).overrides(props).buildInjector();
+      deviceRunning = injector.getInstance(Key.get(new TypeLiteral<Predicate<String>>(){},
+            Names.named(TIMEOUT_NODE_RUNNING)));
+      deviceTerminated = injector.getInstance(Key.get(new TypeLiteral<Predicate<String>>(){},
+              Names.named(TIMEOUT_NODE_TERMINATED)));
+      return injector.getInstance(PacketApi.class);
+   }
+
+   protected void assertNodeRunning(String deviceId) {
+      assertTrue(deviceRunning.apply(deviceId), String.format("Device %s did not start in the configured timeout", deviceId));
+   }
+
+   protected void assertNodeTerminated(String deviceId) {
+      assertTrue(deviceTerminated.apply(deviceId), String.format("Device %s was not terminated in the configured timeout", deviceId));
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiMockTest.java b/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiMockTest.java
new file mode 100644
index 0000000..5b8d6ae
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiMockTest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.internal;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.jclouds.ContextBuilder;
+import org.jclouds.concurrent.config.ExecutorServiceModule;
+import org.jclouds.json.Json;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.PacketProviderMetadata;
+import org.jclouds.rest.ApiContext;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.io.Resources;
+import com.google.common.reflect.TypeToken;
+import com.google.gson.JsonParser;
+import com.google.inject.Module;
+import com.squareup.okhttp.mockwebserver.MockResponse;
+import com.squareup.okhttp.mockwebserver.MockWebServer;
+import com.squareup.okhttp.mockwebserver.RecordedRequest;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static com.google.common.util.concurrent.MoreExecutors.sameThreadExecutor;
+import static org.jclouds.Constants.PROPERTY_MAX_RETRIES;
+import static org.testng.Assert.assertEquals;
+
+public class BasePacketApiMockTest {
+   
+   private static final String X_AUTHORIZATION_TOKEN = "c5401990f0c24135e8d6b5d260603fc71696d4738da9aa04a720229a01a2521d";
+   private static final String DEFAULT_ENDPOINT = new PacketProviderMetadata().getEndpoint();
+   
+   private final Set<Module> modules = ImmutableSet.<Module> of(new ExecutorServiceModule(sameThreadExecutor()));
+   
+   protected MockWebServer server;
+   protected PacketApi api;
+   private Json json;
+   
+   // So that we can ignore formatting.
+   private final JsonParser parser = new JsonParser();
+   
+   @BeforeMethod
+   public void start() throws IOException {
+      server = new MockWebServer();
+      server.play();
+      ApiContext<PacketApi> ctx = ContextBuilder.newBuilder("packet")
+            .credentials("", X_AUTHORIZATION_TOKEN)
+            .endpoint(url(""))
+            .modules(modules)
+            .overrides(overrides())
+            .build();
+      json = ctx.utils().injector().getInstance(Json.class);
+      api = ctx.getApi();
+   }
+
+   @AfterMethod(alwaysRun = true)
+   public void stop() throws IOException {
+      server.shutdown();
+      api.close();
+   }
+   
+   protected Properties overrides() {
+      Properties properties = new Properties();
+      properties.put(PROPERTY_MAX_RETRIES, "0"); // Do not retry
+      return properties;
+   }
+
+   protected String url(String path) {
+      return server.getUrl(path).toString();
+   }
+
+   protected MockResponse jsonResponse(String resource) {
+      return new MockResponse().addHeader("Content-Type", "application/json").setBody(stringFromResource(resource));
+   }
+
+   protected MockResponse response404() {
+      return new MockResponse().setStatus("HTTP/1.1 404 Not Found");
+   }
+   
+   protected MockResponse response204() {
+      return new MockResponse().setStatus("HTTP/1.1 204 No Content");
+   }
+
+   protected String stringFromResource(String resourceName) {
+      try {
+         return Resources.toString(getClass().getResource(resourceName), Charsets.UTF_8)
+               .replace(DEFAULT_ENDPOINT, url(""));
+      } catch (IOException e) {
+         throw Throwables.propagate(e);
+      }
+   }
+   
+   protected <T> T onlyObjectFromResource(String resourceName, TypeToken<Map<String, T>> type) {
+      // Assume JSON objects passed here will be in the form: { "entity": { ... } }
+      String text = stringFromResource(resourceName);
+      Map<String, T> object = json.fromJson(text, type.getType());
+      checkArgument(!object.isEmpty(), "The given json does not contain any object: %s", text);
+      checkArgument(object.keySet().size() == 1, "The given json does not contain more than one object: %s", text);
+      return object.get(getOnlyElement(object.keySet()));
+   }
+   
+   protected <T> T objectFromResource(String resourceName, Class<T> type) {
+      String text = stringFromResource(resourceName);
+      return json.fromJson(text, type);
+   }
+
+   protected RecordedRequest assertSent(MockWebServer server, String method, String path) throws InterruptedException {
+      RecordedRequest request = server.takeRequest();
+      assertEquals(request.getMethod(), method);
+      assertEquals(request.getPath(), path);
+      assertEquals(request.getHeader("Accept"), "application/json");
+      assertEquals(request.getHeader("X-Auth-Token"), X_AUTHORIZATION_TOKEN);
+      return request;
+   }
+
+   protected RecordedRequest assertSent(MockWebServer server, String method, String path, String json)
+         throws InterruptedException {
+      RecordedRequest request = assertSent(server, method, path);
+      assertEquals(request.getHeader("Content-Type"), "application/json");
+      assertEquals(parser.parse(new String(request.getBody(), Charsets.UTF_8)), parser.parse(json));
+      return request;
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/java/org/jclouds/packet/features/ProjectApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/ProjectApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/features/ProjectApiLiveTest.java
new file mode 100644
index 0000000..133e5ef
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/ProjectApiLiveTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jclouds.packet.compute.internal.BasePacketApiLiveTest;
+import org.jclouds.packet.domain.Project;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertTrue;
+import static org.testng.util.Strings.isNullOrEmpty;
+
+@Test(groups = "live", testName = "ProjectApiLiveTest")
+public class ProjectApiLiveTest extends BasePacketApiLiveTest {
+
+   public void testListProjects() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(Iterables.all(api().list().concat(), new Predicate<Project>() {
+         @Override
+         public boolean apply(Project input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All projects must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some projects to be returned");
+   }
+   
+   public void testListActionsOnePage() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(api().list(page(1).perPage(5)).allMatch(new Predicate<Project>() {
+         @Override
+         public boolean apply(Project input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All projects must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some projects to be returned");
+   }
+   
+
+   private ProjectApi api() {
+      return api.projectApi();
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/java/org/jclouds/packet/features/ProjectApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/ProjectApiMockTest.java b/packet/src/test/java/org/jclouds/packet/features/ProjectApiMockTest.java
new file mode 100644
index 0000000..2899020
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/ProjectApiMockTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import org.jclouds.packet.compute.internal.BasePacketApiMockTest;
+import org.jclouds.packet.domain.Project;
+import org.testng.annotations.Test;
+
+import static com.google.common.collect.Iterables.isEmpty;
+import static com.google.common.collect.Iterables.size;
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+@Test(groups = "unit", testName = "ProjectApiMockTest", singleThreaded = true)
+public class ProjectApiMockTest extends BasePacketApiMockTest {
+
+   public void testListProjects() throws InterruptedException {
+      server.enqueue(jsonResponse("/projects-first.json"));
+      server.enqueue(jsonResponse("/projects-last.json"));
+
+      Iterable<Project> projects = api.projectApi().list().concat();
+
+      assertEquals(size(projects), 8); // Force the PagedIterable to advance
+      assertEquals(server.getRequestCount(), 2);
+
+      assertSent(server, "GET", "/projects");
+      assertSent(server, "GET", "/projects?page=2");
+   }
+
+   public void testListProjectsReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<Project> projects = api.projectApi().list().concat();
+
+      assertTrue(isEmpty(projects));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/projects");
+   }
+
+   public void testListProjectsWithOptions() throws InterruptedException {
+      server.enqueue(jsonResponse("/projects-first.json"));
+
+      Iterable<Project> actions = api.projectApi().list(page(1).perPage(5));
+
+      assertEquals(size(actions), 5);
+      assertEquals(server.getRequestCount(), 1);
+
+      assertSent(server, "GET", "/projects?page=1&per_page=5");
+   }
+
+   public void testListProjectsWithOptionsReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<Project> actions = api.projectApi().list(page(1).perPage(5));
+
+      assertTrue(isEmpty(actions));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/projects?page=1&per_page=5");
+   }
+   
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/java/org/jclouds/packet/functions/LinkToListOptionsTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/functions/LinkToListOptionsTest.java b/packet/src/test/java/org/jclouds/packet/functions/LinkToListOptionsTest.java
new file mode 100644
index 0000000..15262e8
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/functions/LinkToListOptionsTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.functions;
+
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.options.ListOptions;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Multimap;
+
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static org.jclouds.packet.domain.options.ListOptions.PAGE_PARAM;
+import static org.jclouds.packet.domain.options.ListOptions.PER_PAGE_PARAM;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
+
+@Test(groups = "unit", testName = "LinkToListOptionsTest")
+public class LinkToListOptionsTest {
+
+   public void testNoOptions() {
+      LinkToListOptions function = new LinkToListOptions();
+
+      ListOptions options = function.apply(Href.create("https://api.packet.net/projects"));
+      assertNotNull(options);
+
+      Multimap<String, String> params = options.buildQueryParameters();
+      assertFalse(params.containsKey(PAGE_PARAM));
+      assertFalse(params.containsKey(PER_PAGE_PARAM));
+   }
+
+   public void testWithOptions() {
+      LinkToListOptions function = new LinkToListOptions();
+
+      ListOptions options = function.apply(Href.create("https://api.packet.net/projects?page=2&per_page=5"));
+      assertNotNull(options);
+
+      Multimap<String, String> params = options.buildQueryParameters();
+      assertEquals(getOnlyElement(params.get(PAGE_PARAM)), "2");
+      assertEquals(getOnlyElement(params.get(PER_PAGE_PARAM)), "5");
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/logback-test.xml b/packet/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..cb55d49
--- /dev/null
+++ b/packet/src/test/resources/logback-test.xml
@@ -0,0 +1,42 @@
+<?xml version="1.0"?>
+<configuration scan="false">
+    <appender name="FILE" class="ch.qos.logback.core.FileAppender">
+        <file>target/test-data/jclouds.log</file>
+        <encoder>
+            <Pattern>%d %-5p [%c] [%thread] %m%n</Pattern>
+        </encoder>
+    </appender>
+    <appender name="WIREFILE" class="ch.qos.logback.core.FileAppender">
+        <file>target/test-data/jclouds-wire.log</file>
+        <encoder>
+            <Pattern>%d %-5p [%c] [%thread] %m%n</Pattern>
+        </encoder>
+    </appender>
+    <appender name="COMPUTEFILE" class="ch.qos.logback.core.FileAppender">
+        <file>target/jclouds-compute.log</file>
+        <encoder>
+            <Pattern>%d %-5p [%c] [%thread] %m%n</Pattern>
+        </encoder>
+    </appender>
+
+    <logger name="org.jclouds">
+        <level value="DEBUG" />
+        <appender-ref ref="FILE" />
+    </logger>
+    <logger name="jclouds.compute">
+        <level value="DEBUG" />
+        <appender-ref ref="COMPUTEFILE" />
+    </logger>
+    <logger name="jclouds.wire">
+        <level value="DEBUG" />
+        <appender-ref ref="WIREFILE" />
+    </logger>
+    <logger name="jclouds.headers">
+        <level value="DEBUG" />
+        <appender-ref ref="WIREFILE" />
+    </logger>
+
+    <root>
+        <level value="INFO" />
+    </root>
+</configuration>

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/resources/projects-first.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/projects-first.json b/packet/src/test/resources/projects-first.json
new file mode 100644
index 0000000..f65d8e3
--- /dev/null
+++ b/packet/src/test/resources/projects-first.json
@@ -0,0 +1,315 @@
+{
+  "projects": [
+    {
+      "id": "93907f48-adfe-43ed-ad89-0e6e83721a54",
+      "name": "Cloudsoft CCS Testing",
+      "created_at": "2016-09-15T08:50:58Z",
+      "updated_at": "2017-01-05T09:36:53Z",
+      "max_devices": {
+        "baremetal_0": null,
+        "baremetal_1": null,
+        "baremetal_2": null,
+        "baremetal_3": null,
+        "baremetal_2a": null,
+        "storage_1": null,
+        "storage_2": null
+      },
+      "members": [
+        {
+          "href": "/users/1140617d-262d-4502-a3d6-771d83c930da"
+        },
+        {
+          "href": "/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"
+        },
+        {
+          "href": "/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"
+        },
+        {
+          "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+        }
+      ],
+      "memberships": [
+        {
+          "href": "/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"
+        },
+        {
+          "href": "/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"
+        },
+        {
+          "href": "/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"
+        },
+        {
+          "href": "/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"
+        }
+      ],
+      "invitations": [],
+      "payment_method": {
+        "href": "/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"
+      },
+      "devices": [
+        {
+          "href": "/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"
+        }
+      ],
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/070e3282-5b6a-4f75-8f18-a4e7488eafaa"
+        }
+      ],
+      "volumes": [],
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+    },
+    {
+      "id": "93907f48-adfe-43ed-ad89-0e6e83721a53",
+      "name": "Cloudsoft CCS Testing",
+      "created_at": "2016-09-15T08:50:58Z",
+      "updated_at": "2017-01-05T09:36:53Z",
+      "max_devices": {
+        "baremetal_0": null,
+        "baremetal_1": null,
+        "baremetal_2": null,
+        "baremetal_3": null,
+        "baremetal_2a": null,
+        "storage_1": null,
+        "storage_2": null
+      },
+      "members": [
+        {
+          "href": "/users/1140617d-262d-4502-a3d6-771d83c930da"
+        },
+        {
+          "href": "/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"
+        },
+        {
+          "href": "/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"
+        },
+        {
+          "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+        }
+      ],
+      "memberships": [
+        {
+          "href": "/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"
+        },
+        {
+          "href": "/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"
+        },
+        {
+          "href": "/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"
+        },
+        {
+          "href": "/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"
+        }
+      ],
+      "invitations": [],
+      "payment_method": {
+        "href": "/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"
+      },
+      "devices": [
+        {
+          "href": "/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"
+        }
+      ],
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/070e3282-5b6a-4f75-8f18-a4e7488eafaa"
+        }
+      ],
+      "volumes": [],
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+    },
+    {
+      "id": "93907f48-adfe-43ed-ad89-0e6e83721a52",
+      "name": "Cloudsoft CCS Testing",
+      "created_at": "2016-09-15T08:50:58Z",
+      "updated_at": "2017-01-05T09:36:53Z",
+      "max_devices": {
+        "baremetal_0": null,
+        "baremetal_1": null,
+        "baremetal_2": null,
+        "baremetal_3": null,
+        "baremetal_2a": null,
+        "storage_1": null,
+        "storage_2": null
+      },
+      "members": [
+        {
+          "href": "/users/1140617d-262d-4502-a3d6-771d83c930da"
+        },
+        {
+          "href": "/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"
+        },
+        {
+          "href": "/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"
+        },
+        {
+          "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+        }
+      ],
+      "memberships": [
+        {
+          "href": "/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"
+        },
+        {
+          "href": "/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"
+        },
+        {
+          "href": "/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"
+        },
+        {
+          "href": "/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"
+        }
+      ],
+      "invitations": [],
+      "payment_method": {
+        "href": "/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"
+      },
+      "devices": [
+        {
+          "href": "/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"
+        }
+      ],
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/070e3282-5b6a-4f75-8f18-a4e7488eafaa"
+        }
+      ],
+      "volumes": [],
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+    },
+    {
+      "id": "93907f48-adfe-43ed-ad89-0e6e83721a51",
+      "name": "Cloudsoft CCS Testing",
+      "created_at": "2016-09-15T08:50:58Z",
+      "updated_at": "2017-01-05T09:36:53Z",
+      "max_devices": {
+        "baremetal_0": null,
+        "baremetal_1": null,
+        "baremetal_2": null,
+        "baremetal_3": null,
+        "baremetal_2a": null,
+        "storage_1": null,
+        "storage_2": null
+      },
+      "members": [
+        {
+          "href": "/users/1140617d-262d-4502-a3d6-771d83c930da"
+        },
+        {
+          "href": "/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"
+        },
+        {
+          "href": "/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"
+        },
+        {
+          "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+        }
+      ],
+      "memberships": [
+        {
+          "href": "/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"
+        },
+        {
+          "href": "/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"
+        },
+        {
+          "href": "/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"
+        },
+        {
+          "href": "/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"
+        }
+      ],
+      "invitations": [],
+      "payment_method": {
+        "href": "/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"
+      },
+      "devices": [
+        {
+          "href": "/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"
+        }
+      ],
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/070e3282-5b6a-4f75-8f18-a4e7488eafaa"
+        }
+      ],
+      "volumes": [],
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+    },
+    {
+      "id": "93907f48-adfe-43ed-ad89-0e6e83721a50",
+      "name": "Cloudsoft CCS Testing",
+      "created_at": "2016-09-15T08:50:58Z",
+      "updated_at": "2017-01-05T09:36:53Z",
+      "max_devices": {
+        "baremetal_0": null,
+        "baremetal_1": null,
+        "baremetal_2": null,
+        "baremetal_3": null,
+        "baremetal_2a": null,
+        "storage_1": null,
+        "storage_2": null
+      },
+      "members": [
+        {
+          "href": "/users/1140617d-262d-4502-a3d6-771d83c930da"
+        },
+        {
+          "href": "/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"
+        },
+        {
+          "href": "/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"
+        },
+        {
+          "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+        }
+      ],
+      "memberships": [
+        {
+          "href": "/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"
+        },
+        {
+          "href": "/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"
+        },
+        {
+          "href": "/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"
+        },
+        {
+          "href": "/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"
+        }
+      ],
+      "invitations": [],
+      "payment_method": {
+        "href": "/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"
+      },
+      "devices": [
+        {
+          "href": "/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"
+        }
+      ],
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/070e3282-5b6a-4f75-8f18-a4e7488eafaa"
+        }
+      ],
+      "volumes": [],
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/projects?page=1"
+    },
+    "previous": null,
+    "self": {
+      "href": "/projects?page=1"
+    },
+    "next": {
+      "href": "/projects?page=2"
+    },
+    "last": {
+      "href": "/projects?page=2"
+    },
+    "total": 8
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/resources/projects-last.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/projects-last.json b/packet/src/test/resources/projects-last.json
new file mode 100644
index 0000000..3d44e67
--- /dev/null
+++ b/packet/src/test/resources/projects-last.json
@@ -0,0 +1,197 @@
+{
+  "projects": [
+    {
+      "id": "93907f48-adfe-43ed-ad89-0e6e83721a55",
+      "name": "Cloudsoft CCS Testing",
+      "created_at": "2016-09-15T08:50:58Z",
+      "updated_at": "2017-01-05T09:36:53Z",
+      "max_devices": {
+        "baremetal_0": null,
+        "baremetal_1": null,
+        "baremetal_2": null,
+        "baremetal_3": null,
+        "baremetal_2a": null,
+        "storage_1": null,
+        "storage_2": null
+      },
+      "members": [
+        {
+          "href": "/users/1140617d-262d-4502-a3d6-771d83c930da"
+        },
+        {
+          "href": "/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"
+        },
+        {
+          "href": "/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"
+        },
+        {
+          "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+        }
+      ],
+      "memberships": [
+        {
+          "href": "/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"
+        },
+        {
+          "href": "/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"
+        },
+        {
+          "href": "/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"
+        },
+        {
+          "href": "/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"
+        }
+      ],
+      "invitations": [],
+      "payment_method": {
+        "href": "/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"
+      },
+      "devices": [
+        {
+          "href": "/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"
+        }
+      ],
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/070e3282-5b6a-4f75-8f18-a4e7488eafaa"
+        }
+      ],
+      "volumes": [],
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+    },
+    {
+      "id": "93907f48-adfe-43ed-ad89-0e6e83721a56",
+      "name": "Cloudsoft CCS Testing",
+      "created_at": "2016-09-15T08:50:58Z",
+      "updated_at": "2017-01-05T09:36:53Z",
+      "max_devices": {
+        "baremetal_0": null,
+        "baremetal_1": null,
+        "baremetal_2": null,
+        "baremetal_3": null,
+        "baremetal_2a": null,
+        "storage_1": null,
+        "storage_2": null
+      },
+      "members": [
+        {
+          "href": "/users/1140617d-262d-4502-a3d6-771d83c930da"
+        },
+        {
+          "href": "/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"
+        },
+        {
+          "href": "/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"
+        },
+        {
+          "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+        }
+      ],
+      "memberships": [
+        {
+          "href": "/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"
+        },
+        {
+          "href": "/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"
+        },
+        {
+          "href": "/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"
+        },
+        {
+          "href": "/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"
+        }
+      ],
+      "invitations": [],
+      "payment_method": {
+        "href": "/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"
+      },
+      "devices": [
+        {
+          "href": "/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"
+        }
+      ],
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/070e3282-5b6a-4f75-8f18-a4e7488eafaa"
+        }
+      ],
+      "volumes": [],
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+    },
+    {
+      "id": "93907f48-adfe-43ed-ad89-0e6e83721a57",
+      "name": "Cloudsoft CCS Testing",
+      "created_at": "2016-09-15T08:50:58Z",
+      "updated_at": "2017-01-05T09:36:53Z",
+      "max_devices": {
+        "baremetal_0": null,
+        "baremetal_1": null,
+        "baremetal_2": null,
+        "baremetal_3": null,
+        "baremetal_2a": null,
+        "storage_1": null,
+        "storage_2": null
+      },
+      "members": [
+        {
+          "href": "/users/1140617d-262d-4502-a3d6-771d83c930da"
+        },
+        {
+          "href": "/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"
+        },
+        {
+          "href": "/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"
+        },
+        {
+          "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+        }
+      ],
+      "memberships": [
+        {
+          "href": "/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"
+        },
+        {
+          "href": "/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"
+        },
+        {
+          "href": "/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"
+        },
+        {
+          "href": "/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"
+        }
+      ],
+      "invitations": [],
+      "payment_method": {
+        "href": "/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"
+      },
+      "devices": [
+        {
+          "href": "/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"
+        }
+      ],
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/070e3282-5b6a-4f75-8f18-a4e7488eafaa"
+        }
+      ],
+      "volumes": [],
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/projects?page=1"
+    },
+    "previous": {
+      "href": "/projects?page=1"
+    },
+    "self": {
+      "href": "/projects?page=2"
+    },
+    "next": null,
+    "last": {
+      "href": "/projects?page=2"
+    },
+    "total": 8
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/b7f39234/packet/src/test/resources/projects.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/projects.json b/packet/src/test/resources/projects.json
new file mode 100644
index 0000000..f7f1710
--- /dev/null
+++ b/packet/src/test/resources/projects.json
@@ -0,0 +1 @@
+{"projects":[{"id":"93907f48-adfe-43ed-ad89-0e6e83721a54","name":"Cloudsoft CCS Testing","created_at":"2016-09-15T08:50:58Z","updated_at":"2017-01-05T09:36:53Z","max_devices":{"baremetal_0":null,"baremetal_1":null,"baremetal_2":null,"baremetal_3":null,"baremetal_2a":null,"storage_1":null,"storage_2":null},"members":[{"href":"/users/1140617d-262d-4502-a3d6-771d83c930da"},{"href":"/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"},{"href":"/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"},{"href":"/users/ad711bc3-6333-449a-a405-23ca81f38c00"}],"memberships":[{"href":"/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"},{"href":"/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"},{"href":"/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"},{"href":"/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"}],"invitations":[],"payment_method":{"href":"/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"},"devices":[{"href":"/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"}],"ssh_keys":[{"href":"/ssh-keys
 /070e3282-5b6a-4f75-8f18-a4e7488eafaa"}],"volumes":[],"href":"/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"}],"meta":{"first":{"href":"/projects?page=1"},"previous":null,"self":{"href":"/projects?page=1"},"next":null,"last":{"href":"/projects?page=1"},"total":1}}


[6/9] jclouds-labs git commit: Add remaining features to support the abstraction

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/operatingSystems.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/operatingSystems.json b/packet/src/test/resources/operatingSystems.json
new file mode 100644
index 0000000..4fd7645
--- /dev/null
+++ b/packet/src/test/resources/operatingSystems.json
@@ -0,0 +1,166 @@
+{
+  "operating_systems": [
+    {
+      "id": "06e21644-a769-11e6-80f5-76304dec7eb7",
+      "slug": "alpine_3",
+      "name": "Alpine 3",
+      "distro": "alpine",
+      "version": "3",
+      "provisionable_on": []
+    },
+    {
+      "id": "06e21978-a769-11e6-80f5-76304dec7eb7",
+      "slug": "centos_6",
+      "name": "CentOS 6",
+      "distro": "centos",
+      "version": "6",
+      "provisionable_on": []
+    },
+    {
+      "id": "68bad60d-f5a7-45c2-ad09-573edaad3a3c",
+      "slug": "centos_7",
+      "name": "Centos 7",
+      "distro": "centos",
+      "version": "7",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e21ce8-a769-11e6-80f5-76304dec7eb7",
+      "slug": "coreos_alpha",
+      "name": "CoreOS Alpha",
+      "distro": "coreos",
+      "version": "alpha",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "6345f310-0bb0-4b59-b051-954fed05183a",
+      "slug": "coreos_beta",
+      "name": "CoreOS Beta",
+      "distro": "coreos",
+      "version": "beta",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "d61c3912-8422-4daf-835e-854efa0062e4",
+      "slug": "coreos_stable",
+      "name": "CoreOS Stable",
+      "distro": "coreos",
+      "version": "stable",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e21e78-a769-11e6-80f5-76304dec7eb7",
+      "slug": "debian_7",
+      "name": "Debian 7",
+      "distro": "debian",
+      "version": "7",
+      "provisionable_on": []
+    },
+    {
+      "id": "88239019-abc7-41e8-9a4d-cd334da97ff1",
+      "slug": "debian_8",
+      "name": "Debian 8",
+      "distro": "debian",
+      "version": "8",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e21ffe-a769-11e6-80f5-76304dec7eb7",
+      "slug": "deprovision",
+      "name": "Deprovision",
+      "distro": "centos",
+      "version": "",
+      "provisionable_on": []
+    },
+    {
+      "id": "06e22198-a769-11e6-80f5-76304dec7eb7",
+      "slug": "freebsd_10_3",
+      "name": "FreeBSD 10.3",
+      "distro": "freebsd",
+      "version": "10.3",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e22328-a769-11e6-80f5-76304dec7eb7",
+      "slug": "rancher",
+      "name": "RancherOS",
+      "distro": "rancher",
+      "version": "latest",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e224b8-a769-11e6-80f5-76304dec7eb7",
+      "slug": "ubuntu_14_04",
+      "name": "Ubuntu 14.04 LTS",
+      "distro": "ubuntu",
+      "version": "14.04",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+      "slug": "ubuntu_16_04",
+      "name": "Ubuntu 16.04 LTS",
+      "distro": "ubuntu",
+      "version": "16.04",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_2a",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e22972-a769-11e6-80f5-76304dec7eb7",
+      "slug": "windows_2012_rc2",
+      "name": "Windows 2012 RC2",
+      "distro": "windows",
+      "version": "2012 RC2",
+      "provisionable_on": [
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/plans-first.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/plans-first.json b/packet/src/test/resources/plans-first.json
new file mode 100644
index 0000000..cbd502a
--- /dev/null
+++ b/packet/src/test/resources/plans-first.json
@@ -0,0 +1,222 @@
+{
+  "plans": [
+    {
+      "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+      "slug": "baremetal_0",
+      "name": "Type 0",
+      "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 1,
+            "type": "Intel Atom C2550 @ 2.4Ghz"
+          }
+        ],
+        "memory": {
+          "total": "8GB"
+        },
+        "drives": [
+          {
+            "count": 1,
+            "size": "80GB",
+            "type": "SSD"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "1Gbps"
+          }
+        ],
+        "features": {
+          "raid": false,
+          "txt": true
+        }
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+        },
+        {
+          "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 0.05
+      }
+    },
+    {
+      "id": "6d1f1ffa-7912-4b78-b50d-88cc7c8ab40f",
+      "slug": "baremetal_1",
+      "name": "Type 1",
+      "description": "Our Type 1 configuration is a zippy general use server, with an Intel E3-1240 v3 processor and 32GB of RAM.",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 1,
+            "type": "Intel E3-1240 v3"
+          }
+        ],
+        "memory": {
+          "total": "32GB"
+        },
+        "drives": [
+          {
+            "count": 2,
+            "size": "120GB",
+            "type": "SSD"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "1Gbps"
+          }
+        ],
+        "features": {
+          "raid": true,
+          "txt": true
+        }
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+        },
+        {
+          "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 0.4
+      }
+    },
+    {
+      "id": "a3729923-fdc4-4e85-a972-aafbad3695db",
+      "slug": "baremetal_2",
+      "name": "Type 2",
+      "description": "Our Type 2 configuration is the perfect all purpose virtualization server, with dual E5-2650 v4 processors, 256 GB of DDR4 RAM, and six SSDs totaling 2.8 TB of storage.",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 2,
+            "type": "Intel Xeon E5-2650 v4 @2.2GHz"
+          }
+        ],
+        "memory": {
+          "total": "256GB"
+        },
+        "drives": [
+          {
+            "count": 6,
+            "size": "480GB",
+            "type": "SSD"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "10Gbps"
+          }
+        ],
+        "features": {
+          "raid": true,
+          "txt": true
+        }
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 1.25
+      }
+    },
+    {
+      "id": "3bc8a214-b807-4058-ad4a-6925f2411155",
+      "slug": "baremetal_2a",
+      "name": "Type 2A",
+      "description": "Our Type 2A configuration is a 96-core dual socket ARM 64 beast based on Cavium ThunderX chips",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 2,
+            "type": "Cavium ThunderX CN8890 @2GHz"
+          }
+        ],
+        "memory": {
+          "total": "128GB"
+        },
+        "drives": [
+          {
+            "count": 1,
+            "size": "512GB",
+            "type": "SSD"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "10Gbps"
+          }
+        ],
+        "features": {}
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+        },
+        {
+          "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 0.5
+      }
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/plans?page=1"
+    },
+    "previous": null,
+    "self": {
+      "href": "/plans?page=1"
+    },
+    "next": {
+      "href": "/plans?page=2"
+    },
+    "last": {
+      "href": "/plans?page=2"
+    },
+    "total": 7
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/plans-last.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/plans-last.json b/packet/src/test/resources/plans-last.json
new file mode 100644
index 0000000..04d6aee
--- /dev/null
+++ b/packet/src/test/resources/plans-last.json
@@ -0,0 +1,98 @@
+{
+  "plans": [
+    {
+      "id": "741f3afb-bb2f-4694-93a0-fcbad7cd5e78",
+      "slug": "baremetal_3",
+      "name": "Type 3",
+      "description": "Our Type 3 configuration is a high core, high IO server, with dual Intel E5-2640 v3 processors, 128GB of DDR4 RAM and ultra fast NVME flash drives.",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 2,
+            "type": "Intel E5-2640 v3"
+          }
+        ],
+        "memory": {
+          "total": "128GB"
+        },
+        "drives": [
+          {
+            "count": 2,
+            "size": "120GB",
+            "type": "SSD"
+          },
+          {
+            "count": 1,
+            "size": "1.6TB",
+            "type": "NVME"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "10Gbps"
+          }
+        ],
+        "features": {
+          "raid": true,
+          "txt": true
+        }
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 1.75
+      }
+    },
+    {
+      "id": "87728148-3155-4992-a730-8d1e6aca8a32",
+      "slug": "storage_1",
+      "name": "Standard",
+      "description": "TBD",
+      "line": "storage",
+      "specs": {},
+      "available_in": [],
+      "pricing": {
+        "hour": 0.000104
+      }
+    },
+    {
+      "id": "d6570cfb-38fa-4467-92b3-e45d059bb249",
+      "slug": "storage_2",
+      "name": "Performance",
+      "description": "TBD",
+      "line": "storage",
+      "specs": {},
+      "available_in": [],
+      "pricing": {
+        "hour": 0.000223
+      }
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/plans?page=1"
+    },
+    "previous": {
+      "href": "/plans?page=1"
+    },
+    "self": {
+      "href": "/plans?page=2"
+    },
+    "next": null,
+    "last": {
+      "href": "/plans?page=2"
+    },
+    "total": 7
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/plans.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/plans.json b/packet/src/test/resources/plans.json
new file mode 100644
index 0000000..c3eeedd
--- /dev/null
+++ b/packet/src/test/resources/plans.json
@@ -0,0 +1,284 @@
+{
+  "plans": [
+    {
+      "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+      "slug": "baremetal_0",
+      "name": "Type 0",
+      "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 1,
+            "type": "Intel Atom C2550 @ 2.4Ghz"
+          }
+        ],
+        "memory": {
+          "total": "8GB"
+        },
+        "drives": [
+          {
+            "count": 1,
+            "size": "80GB",
+            "type": "SSD"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "1Gbps"
+          }
+        ],
+        "features": {
+          "raid": false,
+          "txt": true
+        }
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+        },
+        {
+          "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 0.05
+      }
+    },
+    {
+      "id": "6d1f1ffa-7912-4b78-b50d-88cc7c8ab40f",
+      "slug": "baremetal_1",
+      "name": "Type 1",
+      "description": "Our Type 1 configuration is a zippy general use server, with an Intel E3-1240 v3 processor and 32GB of RAM.",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 1,
+            "type": "Intel E3-1240 v3"
+          }
+        ],
+        "memory": {
+          "total": "32GB"
+        },
+        "drives": [
+          {
+            "count": 2,
+            "size": "120GB",
+            "type": "SSD"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "1Gbps"
+          }
+        ],
+        "features": {
+          "raid": true,
+          "txt": true
+        }
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+        },
+        {
+          "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 0.4
+      }
+    },
+    {
+      "id": "a3729923-fdc4-4e85-a972-aafbad3695db",
+      "slug": "baremetal_2",
+      "name": "Type 2",
+      "description": "Our Type 2 configuration is the perfect all purpose virtualization server, with dual E5-2650 v4 processors, 256 GB of DDR4 RAM, and six SSDs totaling 2.8 TB of storage.",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 2,
+            "type": "Intel Xeon E5-2650 v4 @2.2GHz"
+          }
+        ],
+        "memory": {
+          "total": "256GB"
+        },
+        "drives": [
+          {
+            "count": 6,
+            "size": "480GB",
+            "type": "SSD"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "10Gbps"
+          }
+        ],
+        "features": {
+          "raid": true,
+          "txt": true
+        }
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 1.25
+      }
+    },
+    {
+      "id": "3bc8a214-b807-4058-ad4a-6925f2411155",
+      "slug": "baremetal_2a",
+      "name": "Type 2A",
+      "description": "Our Type 2A configuration is a 96-core dual socket ARM 64 beast based on Cavium ThunderX chips",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 2,
+            "type": "Cavium ThunderX CN8890 @2GHz"
+          }
+        ],
+        "memory": {
+          "total": "128GB"
+        },
+        "drives": [
+          {
+            "count": 1,
+            "size": "512GB",
+            "type": "SSD"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "10Gbps"
+          }
+        ],
+        "features": {}
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+        },
+        {
+          "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 0.5
+      }
+    },
+    {
+      "id": "741f3afb-bb2f-4694-93a0-fcbad7cd5e78",
+      "slug": "baremetal_3",
+      "name": "Type 3",
+      "description": "Our Type 3 configuration is a high core, high IO server, with dual Intel E5-2640 v3 processors, 128GB of DDR4 RAM and ultra fast NVME flash drives.",
+      "line": "baremetal",
+      "specs": {
+        "cpus": [
+          {
+            "count": 2,
+            "type": "Intel E5-2640 v3"
+          }
+        ],
+        "memory": {
+          "total": "128GB"
+        },
+        "drives": [
+          {
+            "count": 2,
+            "size": "120GB",
+            "type": "SSD"
+          },
+          {
+            "count": 1,
+            "size": "1.6TB",
+            "type": "NVME"
+          }
+        ],
+        "nics": [
+          {
+            "count": 2,
+            "type": "10Gbps"
+          }
+        ],
+        "features": {
+          "raid": true,
+          "txt": true
+        }
+      },
+      "available_in": [
+        {
+          "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+        },
+        {
+          "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+        },
+        {
+          "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+        }
+      ],
+      "pricing": {
+        "hour": 1.75
+      }
+    },
+    {
+      "id": "87728148-3155-4992-a730-8d1e6aca8a32",
+      "slug": "storage_1",
+      "name": "Standard",
+      "description": "TBD",
+      "line": "storage",
+      "specs": {},
+      "available_in": [],
+      "pricing": {
+        "hour": 0.000104
+      }
+    },
+    {
+      "id": "d6570cfb-38fa-4467-92b3-e45d059bb249",
+      "slug": "storage_2",
+      "name": "Performance",
+      "description": "TBD",
+      "line": "storage",
+      "specs": {},
+      "available_in": [],
+      "pricing": {
+        "hour": 0.000223
+      }
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/power-off.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/power-off.json b/packet/src/test/resources/power-off.json
new file mode 100644
index 0000000..3cdd43f
--- /dev/null
+++ b/packet/src/test/resources/power-off.json
@@ -0,0 +1,3 @@
+{
+  "type": "power_off"
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/power-on.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/power-on.json b/packet/src/test/resources/power-on.json
new file mode 100644
index 0000000..a482c1c
--- /dev/null
+++ b/packet/src/test/resources/power-on.json
@@ -0,0 +1,3 @@
+{
+  "type": "power_on"
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/projects.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/projects.json b/packet/src/test/resources/projects.json
index f7f1710..548b4af 100644
--- a/packet/src/test/resources/projects.json
+++ b/packet/src/test/resources/projects.json
@@ -1 +1,77 @@
-{"projects":[{"id":"93907f48-adfe-43ed-ad89-0e6e83721a54","name":"Cloudsoft CCS Testing","created_at":"2016-09-15T08:50:58Z","updated_at":"2017-01-05T09:36:53Z","max_devices":{"baremetal_0":null,"baremetal_1":null,"baremetal_2":null,"baremetal_3":null,"baremetal_2a":null,"storage_1":null,"storage_2":null},"members":[{"href":"/users/1140617d-262d-4502-a3d6-771d83c930da"},{"href":"/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"},{"href":"/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"},{"href":"/users/ad711bc3-6333-449a-a405-23ca81f38c00"}],"memberships":[{"href":"/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"},{"href":"/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"},{"href":"/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"},{"href":"/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"}],"invitations":[],"payment_method":{"href":"/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"},"devices":[{"href":"/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"}],"ssh_keys":[{"href":"/ssh-keys
 /070e3282-5b6a-4f75-8f18-a4e7488eafaa"}],"volumes":[],"href":"/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"}],"meta":{"first":{"href":"/projects?page=1"},"previous":null,"self":{"href":"/projects?page=1"},"next":null,"last":{"href":"/projects?page=1"},"total":1}}
+{
+  "projects": [
+    {
+      "id": "93907f48-adfe-43ed-ad89-0e6e83721a54",
+      "name": "Cloudsoft CCS Testing",
+      "created_at": "2016-09-15T08:50:58Z",
+      "updated_at": "2017-01-05T09:36:53Z",
+      "max_devices": {
+        "baremetal_0": null,
+        "baremetal_1": null,
+        "baremetal_2": null,
+        "baremetal_3": null,
+        "baremetal_2a": null,
+        "storage_1": null,
+        "storage_2": null
+      },
+      "members": [
+        {
+          "href": "/users/1140617d-262d-4502-a3d6-771d83c930da"
+        },
+        {
+          "href": "/users/343345fe-18b3-46a3-9b9c-e4a2fe88ccbd"
+        },
+        {
+          "href": "/users/73b0442e-cc4b-42a0-8d3a-c8dfb8a4ff2e"
+        },
+        {
+          "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+        }
+      ],
+      "memberships": [
+        {
+          "href": "/memberships/914facae-547f-46fc-93e8-860eb53d9bf6"
+        },
+        {
+          "href": "/memberships/1ce7c9d9-d11f-47f2-b5a6-d1221338ad69"
+        },
+        {
+          "href": "/memberships/e25c4478-9e44-465a-a9d7-bffc7a83300d"
+        },
+        {
+          "href": "/memberships/8a00c05b-3ddc-41c3-8dd7-4a3d5984ecec"
+        }
+      ],
+      "invitations": [],
+      "payment_method": {
+        "href": "/payment-methods/b2bb4ee0-506b-4c01-b17d-ba41d5c430c5"
+      },
+      "devices": [
+        {
+          "href": "/devices/4942a6c7-55c6-42cc-a06b-cccd6f2fa5df"
+        }
+      ],
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/070e3282-5b6a-4f75-8f18-a4e7488eafaa"
+        }
+      ],
+      "volumes": [],
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/projects?page=1"
+    },
+    "previous": null,
+    "self": {
+      "href": "/projects?page=1"
+    },
+    "next": null,
+    "last": {
+      "href": "/projects?page=1"
+    },
+    "total": 1
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/reboot.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/reboot.json b/packet/src/test/resources/reboot.json
new file mode 100644
index 0000000..9bf6dfd
--- /dev/null
+++ b/packet/src/test/resources/reboot.json
@@ -0,0 +1,3 @@
+{
+  "type": "reboot"
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/rescue.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/rescue.json b/packet/src/test/resources/rescue.json
new file mode 100644
index 0000000..91ea877
--- /dev/null
+++ b/packet/src/test/resources/rescue.json
@@ -0,0 +1,3 @@
+{
+  "type": "rescue"
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/ssh-key-create-req.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/ssh-key-create-req.json b/packet/src/test/resources/ssh-key-create-req.json
new file mode 100644
index 0000000..02bdeff
--- /dev/null
+++ b/packet/src/test/resources/ssh-key-create-req.json
@@ -0,0 +1,4 @@
+{
+  "label": "jclouds-ssh-key-livetest",
+  "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQCdgcoNzH4hCc0j3b4MuG503L/J54uyFvwCAOu8vSsYuLpJ4AEyEOv+T0SfdF605fK6GYXA16Rxk3lrPt7mfKGNtXR0Ripbv7Zc6PvCRorwgj/cjh/45miozjrkXAiHD1GFZycfbi4YsoWAqZj7W4mwtctmhrYM0FPdya2XoRpVy89N+A5Xo4Xtd6EZn6JGEKQM5+kF2aL3ggy0od/DqjuEVYwZoyTe1RgUTXZSU/Woh7WMhsRHbqd3eYz4s6ac8n8IJPGKtUaQeqUtH7OK6NRYXVypUrkqNlwdNYZAwrjXg/x5T3D+bo11LENASRt9OJ2OkmRSTqRxBeDkhnVauWK/"
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/ssh-key-create-res.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/ssh-key-create-res.json b/packet/src/test/resources/ssh-key-create-res.json
new file mode 100644
index 0000000..fa386fb
--- /dev/null
+++ b/packet/src/test/resources/ssh-key-create-res.json
@@ -0,0 +1,12 @@
+{
+  "id": "da5d6c21-2e8c-43ac-820f-ff41bd4e6ebc",
+  "label": "andrea-device-livetest",
+  "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQCdgcoNzH4hCc0j3b4MuG503L/J54uyFvwCAOu8vSsYuLpJ4AEyEOv+T0SfdF605fK6GYXA16Rxk3lrPt7mfKGNtXR0Ripbv7Zc6PvCRorwgj/cjh/45miozjrkXAiHD1GFZycfbi4YsoWAqZj7W4mwtctmhrYM0FPdya2XoRpVy89N+A5Xo4Xtd6EZn6JGEKQM5+kF2aL3ggy0od/DqjuEVYwZoyTe1RgUTXZSU/Woh7WMhsRHbqd3eYz4s6ac8n8IJPGKtUaQeqUtH7OK6NRYXVypUrkqNlwdNYZAwrjXg/x5T3D+bo11LENASRt9OJ2OkmRSTqRxBeDkhnVauWK/",
+  "fingerprint": "60:b0:5a:23:42:b8:53:19:d0:32:f1:4d:a8:7c:9b:71",
+  "created_at": "2017-01-20T14:15:26Z",
+  "updated_at": "2017-01-20T14:15:26Z",
+  "owner": {
+    "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+  },
+  "href": "/ssh-keys/da5d6c21-2e8c-43ac-820f-ff41bd4e6ebc"
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/ssh-key.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/ssh-key.json b/packet/src/test/resources/ssh-key.json
new file mode 100644
index 0000000..f3f21ea
--- /dev/null
+++ b/packet/src/test/resources/ssh-key.json
@@ -0,0 +1,12 @@
+{
+  "id": "bba63e41-b12c-493a-81d4-e52f50f247ed",
+  "label": "test",
+  "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+  "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+  "created_at": "2017-01-03T09:42:54Z",
+  "updated_at": "2017-01-03T09:42:54Z",
+  "owner": {
+    "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+  },
+  "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/sshKeys-first.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/sshKeys-first.json b/packet/src/test/resources/sshKeys-first.json
new file mode 100644
index 0000000..1f0b9b3
--- /dev/null
+++ b/packet/src/test/resources/sshKeys-first.json
@@ -0,0 +1,80 @@
+{
+  "ssh_keys": [
+    {
+      "id": "bba63e41-b12c-493a-81d4-e52f50f247e1",
+      "label": "test",
+      "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+      "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+      "created_at": "2017-01-03T09:42:54Z",
+      "updated_at": "2017-01-03T09:42:54Z",
+      "owner": {
+        "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+      },
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    },
+    {
+      "id": "bba63e41-b12c-493a-81d4-e52f50f247e2",
+      "label": "test",
+      "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+      "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+      "created_at": "2017-01-03T09:42:54Z",
+      "updated_at": "2017-01-03T09:42:54Z",
+      "owner": {
+        "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+      },
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    },
+    {
+      "id": "bba63e41-b12c-493a-81d4-e52f50f247e3",
+      "label": "test",
+      "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+      "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+      "created_at": "2017-01-03T09:42:54Z",
+      "updated_at": "2017-01-03T09:42:54Z",
+      "owner": {
+        "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+      },
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    },
+    {
+      "id": "bba63e41-b12c-493a-81d4-e52f50f247e4",
+      "label": "test",
+      "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+      "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+      "created_at": "2017-01-03T09:42:54Z",
+      "updated_at": "2017-01-03T09:42:54Z",
+      "owner": {
+        "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+      },
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    },
+    {
+      "id": "bba63e41-b12c-493a-81d4-e52f50f247e5",
+      "label": "test",
+      "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+      "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+      "created_at": "2017-01-03T09:42:54Z",
+      "updated_at": "2017-01-03T09:42:54Z",
+      "owner": {
+        "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+      },
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/ssh-keys?page=1"
+    },
+    "previous": null,
+    "self": {
+      "href": "/ssh-keys?page=1"
+    },
+    "next": {
+      "href": "/ssh-keys?page=2"
+    },
+    "last": {
+      "href": "/ssh-keys?page=2"
+    },
+    "total": 8
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/sshKeys-last.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/sshKeys-last.json b/packet/src/test/resources/sshKeys-last.json
new file mode 100644
index 0000000..15fd116
--- /dev/null
+++ b/packet/src/test/resources/sshKeys-last.json
@@ -0,0 +1,56 @@
+{
+  "ssh_keys": [
+    {
+      "id": "bba63e41-b12c-493a-81d4-e52f50f247e6",
+      "label": "test",
+      "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+      "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+      "created_at": "2017-01-03T09:42:54Z",
+      "updated_at": "2017-01-03T09:42:54Z",
+      "owner": {
+        "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+      },
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    },
+    {
+      "id": "bba63e41-b12c-493a-81d4-e52f50f247e7",
+      "label": "test",
+      "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+      "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+      "created_at": "2017-01-03T09:42:54Z",
+      "updated_at": "2017-01-03T09:42:54Z",
+      "owner": {
+        "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+      },
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    },
+    {
+      "id": "bba63e41-b12c-493a-81d4-e52f50f247e8",
+      "label": "test",
+      "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+      "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+      "created_at": "2017-01-03T09:42:54Z",
+      "updated_at": "2017-01-03T09:42:54Z",
+      "owner": {
+        "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+      },
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/ssh-keys?page=1"
+    },
+    "previous": {
+      "href": "/ssh-keys?page=1"
+    },
+    "self": {
+      "href": "/ssh-keys?page=2"
+    },
+    "next": null,
+    "last": {
+      "href": "/ssh-keys?page=2"
+    },
+    "total": 8
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/sshKeys.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/sshKeys.json b/packet/src/test/resources/sshKeys.json
new file mode 100644
index 0000000..582c730
--- /dev/null
+++ b/packet/src/test/resources/sshKeys.json
@@ -0,0 +1,16 @@
+{
+  "ssh_keys": [
+    {
+      "id": "bba63e41-b12c-493a-81d4-e52f50f247ed",
+      "label": "test",
+      "key": "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQDNnvPZ25wZK19grrbal6R5JP1hLRBUuNh86KxZVMAFkGd5ouVSgO9dFajHZ45Q4mbaTkdOiqf7otMLDzkaztEa7oLK7Jso0Y0LOi+nT4gf38rvbEF5mq069G9b9XqlNleaGnpLuTN54iEK8c4TWZxIJqgelHEHhjp7V1asmilBbpZDmwA5cTt9vGJIhqA/BptKH3folZKeAQjRa1ZRSwSQUnk9rBKn4PKSRDojVa9A9jKt4qboJh5q7ZjtE8z+665F/4TgLzElTXUA8+uUFGpuynMSmQEt301e18dXAl+vBr8fMiThcoVVbdVdqdjXsw75fMXPAgqhjrw8k3+0/4P9 andrea@bigmac.local",
+      "fingerprint": "88:8a:7d:0f:db:fb:f4:2a:63:2e:d3:4b:1f:7f:0b:12",
+      "created_at": "2017-01-03T09:42:54Z",
+      "updated_at": "2017-01-03T09:42:54Z",
+      "owner": {
+        "href": "/users/ad711bc3-6333-449a-a405-23ca81f38c00"
+      },
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/user.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/user.json b/packet/src/test/resources/user.json
new file mode 100644
index 0000000..a3c4677
--- /dev/null
+++ b/packet/src/test/resources/user.json
@@ -0,0 +1,19 @@
+{
+  "id": "faaaec30-be0c-4b6d-b1a7-1bf6e494a333",
+  "first_name": "Melissa",
+  "last_name": "Smith",
+  "full_name": "Melissa Smith",
+  "email": "melissa@example.com",
+  "max_projects": 1,
+  "two_factor_auth": "sms",
+  "timezone": "America/New_York",
+  "phone_number": "555-555-5555",
+  "created_at": "2014-04-14T02:15:15Z",
+  "updated_at": "2014-04-14T02:15:15Z",
+  "timezone": "America/New_York",
+  "emails": [
+    { "href": "/emails/7e407a6a-4f96-4da0-8c48-5665352ebf06" }
+  ],
+  "avatar_url": "https://www.gravatar.com/avatar/d248bbb3a03ed?d=https://app.packet.net/client-portal/assets/images/default_gravatar.png",
+  "href": "/users/faaaec30-be0c-4b6d-b1a7-1bf6e494a333"
+}


[8/9] jclouds-labs git commit: Add remaining features to support the abstraction

Posted by na...@apache.org.
Add remaining features to support the abstraction

- add DeviceApi with Mock and Live Test
- add FacilityApi with Mock and Live Test
- add OperatingSystemApi with Mock and Live Test
- add SshKeyApi with Mock and Live Test
- fix Device domain object
- refactor deviceApi.create and sshKeyApi.create as they actually return an object instead of a URI now
- add mock and live tests for device api actions


Project: http://git-wip-us.apache.org/repos/asf/jclouds-labs/repo
Commit: http://git-wip-us.apache.org/repos/asf/jclouds-labs/commit/a812591f
Tree: http://git-wip-us.apache.org/repos/asf/jclouds-labs/tree/a812591f
Diff: http://git-wip-us.apache.org/repos/asf/jclouds-labs/diff/a812591f

Branch: refs/heads/2.0.x
Commit: a812591f2387b2809e404b83e2af8ed07aa6b95d
Parents: 86bfeab
Author: Andrea Turli <an...@gmail.com>
Authored: Wed Jan 18 19:13:28 2017 +0100
Committer: Ignasi Barrera <na...@apache.org>
Committed: Thu Feb 2 15:23:25 2017 +0100

----------------------------------------------------------------------
 .../main/java/org/jclouds/packet/PacketApi.java |  46 +
 .../packet/config/PacketHttpApiModule.java      |   4 +-
 .../java/org/jclouds/packet/domain/Device.java  |  82 +-
 .../packet/domain/ProvisioningEvent.java        |  52 ++
 .../org/jclouds/packet/features/DeviceApi.java  | 148 +++
 .../jclouds/packet/features/FacilityApi.java    |  94 ++
 .../packet/features/OperatingSystemApi.java     |  94 ++
 .../org/jclouds/packet/features/PlanApi.java    |  94 ++
 .../org/jclouds/packet/features/ProjectApi.java |   6 +-
 .../org/jclouds/packet/features/SshKeyApi.java  | 122 +++
 .../packet/functions/BaseToPagedIterable.java   |   8 +-
 .../packet/functions/HrefToListOptions.java     |  63 ++
 .../packet/functions/LinkToListOptions.java     |  63 --
 .../packet/features/DeviceApiLiveTest.java      | 138 +++
 .../packet/features/DeviceApiMockTest.java      | 184 ++++
 .../packet/features/FacilityApiLiveTest.java    |  62 ++
 .../packet/features/FacilityApiMockTest.java    |  78 ++
 .../features/OperatingSystemApiLiveTest.java    |  62 ++
 .../features/OperatingSystemApiMockTest.java    |  78 ++
 .../packet/features/PlanApiLiveTest.java        |  62 ++
 .../packet/features/PlanApiMockTest.java        |  78 ++
 .../packet/features/ProjectApiLiveTest.java     |   7 +-
 .../packet/features/ProjectApiMockTest.java     |   1 -
 .../packet/features/SshKeyApiLiveTest.java      |  83 ++
 .../packet/features/SshKeyApiMockTest.java      | 134 +++
 .../packet/functions/HrefToListOptionsTest.java |  57 ++
 .../packet/functions/LinkToListOptionsTest.java |  57 --
 .../src/test/resources/device-create-req.json   |  11 +
 .../src/test/resources/device-create-res.json   | 211 +++++
 packet/src/test/resources/device.json           | 278 ++++++
 packet/src/test/resources/devices-first.json    | 910 +++++++++++++++++++
 packet/src/test/resources/devices-last.json     | 376 ++++++++
 packet/src/test/resources/devices.json          | 282 ++++++
 packet/src/test/resources/facilities-first.json |  39 +
 packet/src/test/resources/facilities-last.json  |  27 +
 packet/src/test/resources/facilities.json       |  30 +
 .../test/resources/operatingSystems-first.json  |  96 ++
 .../test/resources/operatingSystems-last.json   | 106 +++
 packet/src/test/resources/operatingSystems.json | 166 ++++
 packet/src/test/resources/plans-first.json      | 222 +++++
 packet/src/test/resources/plans-last.json       |  98 ++
 packet/src/test/resources/plans.json            | 284 ++++++
 packet/src/test/resources/power-off.json        |   3 +
 packet/src/test/resources/power-on.json         |   3 +
 packet/src/test/resources/projects.json         |  78 +-
 packet/src/test/resources/reboot.json           |   3 +
 packet/src/test/resources/rescue.json           |   3 +
 .../src/test/resources/ssh-key-create-req.json  |   4 +
 .../src/test/resources/ssh-key-create-res.json  |  12 +
 packet/src/test/resources/ssh-key.json          |  12 +
 packet/src/test/resources/sshKeys-first.json    |  80 ++
 packet/src/test/resources/sshKeys-last.json     |  56 ++
 packet/src/test/resources/sshKeys.json          |  16 +
 packet/src/test/resources/user.json             |  19 +
 54 files changed, 5244 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/PacketApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/PacketApi.java b/packet/src/main/java/org/jclouds/packet/PacketApi.java
index 1cb8e9b..e5f84e3 100644
--- a/packet/src/main/java/org/jclouds/packet/PacketApi.java
+++ b/packet/src/main/java/org/jclouds/packet/PacketApi.java
@@ -18,7 +18,14 @@ package org.jclouds.packet;
 
 import java.io.Closeable;
 
+import javax.ws.rs.PathParam;
+
+import org.jclouds.packet.features.DeviceApi;
+import org.jclouds.packet.features.FacilityApi;
+import org.jclouds.packet.features.OperatingSystemApi;
+import org.jclouds.packet.features.PlanApi;
 import org.jclouds.packet.features.ProjectApi;
+import org.jclouds.packet.features.SshKeyApi;
 import org.jclouds.rest.annotations.Delegate;
 
 /**
@@ -37,4 +44,43 @@ public interface PacketApi extends Closeable {
    @Delegate
    ProjectApi projectApi();
 
+   /**
+    * This Packet API provides all of the devices
+    *
+    * @see <a href="https://www.packet.net/help/api/#page:devices">docs</a>
+    */
+   @Delegate
+   DeviceApi deviceApi(@PathParam("projectId") String projectId);
+
+   /**
+    * This Packet API provides all of the facilities
+    *
+    * @see <a href="https://www.packet.net/help/api/#page:devices,header:devices-operating-systems">docs</a>
+    */
+   @Delegate
+   FacilityApi facilityApi();
+
+   /**
+    * This Packet API provides all of the plans
+    *
+    * @see <a href="https://www.packet.net/help/api/#page:devices,header:devices-plans">docs</a>
+    */
+   @Delegate
+   PlanApi planApi();
+
+   /**
+    * This Packet API provides all of the operating systems
+    *
+    * @see <a href="https://www.packet.net/help/api/#page:devices,header:devices-operating-systems">docs</a>
+    */
+   @Delegate
+   OperatingSystemApi operatingSystemApi();
+
+   /**
+    * This Packet API provides all of the operating systems
+    *
+    * @see <a href="https://www.packet.net/help/api/#page:ssh-keys">docs</a>
+    */
+   @Delegate
+   SshKeyApi sshKeyApi();
 }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/config/PacketHttpApiModule.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/config/PacketHttpApiModule.java b/packet/src/main/java/org/jclouds/packet/config/PacketHttpApiModule.java
index e74bb19..6edeebd 100644
--- a/packet/src/main/java/org/jclouds/packet/config/PacketHttpApiModule.java
+++ b/packet/src/main/java/org/jclouds/packet/config/PacketHttpApiModule.java
@@ -25,7 +25,7 @@ import org.jclouds.location.suppliers.implicit.FirstRegion;
 import org.jclouds.packet.PacketApi;
 import org.jclouds.packet.domain.Href;
 import org.jclouds.packet.domain.options.ListOptions;
-import org.jclouds.packet.functions.LinkToListOptions;
+import org.jclouds.packet.functions.HrefToListOptions;
 import org.jclouds.packet.handlers.PacketErrorHandler;
 import org.jclouds.rest.ConfiguresHttpApi;
 import org.jclouds.rest.config.HttpApiModule;
@@ -42,7 +42,7 @@ public class PacketHttpApiModule extends HttpApiModule<PacketApi> {
       super.configure();
       bind(ImplicitLocationSupplier.class).to(FirstRegion.class).in(Scopes.SINGLETON);
       bind(new TypeLiteral<Function<Href, ListOptions>>() {
-      }).to(LinkToListOptions.class);
+      }).to(HrefToListOptions.class);
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/domain/Device.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Device.java b/packet/src/main/java/org/jclouds/packet/domain/Device.java
index af59970..96e0b53 100644
--- a/packet/src/main/java/org/jclouds/packet/domain/Device.java
+++ b/packet/src/main/java/org/jclouds/packet/domain/Device.java
@@ -18,6 +18,8 @@ package org.jclouds.packet.domain;
 
 import java.util.Date;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.jclouds.javax.annotation.Nullable;
 import org.jclouds.json.SerializedNames;
@@ -27,6 +29,8 @@ import com.google.common.base.Enums;
 import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
@@ -60,15 +64,19 @@ public abstract class Device {
     public abstract OperatingSystem operatingSystem();
     public abstract Facility facility();
     public abstract Href project();
+    public abstract List<Href> sshKeys();
     public abstract Href projectLite();
     public abstract List<Object> volumes();
     public abstract List<IpAddress> ipAddresses();
+    public abstract List<ProvisioningEvent> provisioningEvents();
     public abstract Plan plan();
     public abstract String rootPassword();
     public abstract String userdata();
     public abstract String href();
 
-    @SerializedNames({"id", "short_id", "hostname", "description", "state", "tags", "billing_cycle", "user", "iqn", "locked", "bonding_mode", "created_at", "updated_at", "operating_system", "facility", "project", "project_lite", "volumes", "ip_addresses", "plan", "root_password", "userdata", "href"})
+    @SerializedNames({"id", "short_id", "hostname", "description", "state", "tags", "billing_cycle", "user", "iqn",
+            "locked", "bonding_mode", "created_at", "updated_at", "operating_system", "facility", "project", "ssh_keys",
+            "project_lite", "volumes", "ip_addresses", "provisioning_events", "plan", "root_password", "userdata", "href"})
     public static Device create(String id,
                                 String shortId,
                                 String hostname,
@@ -85,9 +93,11 @@ public abstract class Device {
                                 OperatingSystem operatingSystem,
                                 Facility facility,
                                 Href project,
+                                List<Href> sshKeys,
                                 Href projectLite,
                                 List<Object> volumes,
                                 List<IpAddress> ipAddresses,
+                                List<ProvisioningEvent> provisioningEvents,
                                 Plan plan,
                                 String rootPassword,
                                 String userdata,
@@ -95,14 +105,80 @@ public abstract class Device {
     ) {
         return new AutoValue_Device(id, shortId, hostname, description, state,
                 tags == null ? ImmutableList.<String> of() : ImmutableList.copyOf(tags),
-                billingCycle, user, iqn, locked, bondingMode, createdAt, updatedAt, operatingSystem, facility, project, projectLite,
+                billingCycle, user, iqn, locked, bondingMode, createdAt, updatedAt, operatingSystem, facility, project,
+                sshKeys == null ? ImmutableList.<Href> of() : ImmutableList.copyOf(sshKeys),
+                projectLite,
                 volumes == null ? ImmutableList.of() : ImmutableList.copyOf(volumes),
                 ipAddresses == null ? ImmutableList.<IpAddress>of() : ImmutableList.copyOf(ipAddresses),
-                plan, rootPassword, userdata, href
+                provisioningEvents == null ? ImmutableList.<ProvisioningEvent> of() : ImmutableList.copyOf(provisioningEvents),
+                plan,
+                rootPassword, userdata, href
         );
     }
 
     Device() {
     }
 
+    @AutoValue
+    public abstract static class CreateDevice {
+
+        public abstract String hostname();
+        public abstract String plan();
+        public abstract String billingCycle();
+        public abstract String facility();
+        public abstract Map<String, String> features();
+        public abstract String operatingSystem();
+        public abstract Boolean locked();
+        public abstract String userdata();
+        public abstract Set<String> tags();
+
+        @SerializedNames({"hostname", "plan", "billing_cycle", "facility", "features", "operating_system",
+                "locked", "userdata", "tags" })
+        private static CreateDevice create(final String hostname, final String plan, final String billingCycle,
+                                          final String facility, final Map<String, String> features, final String operatingSystem,
+                                          final Boolean locked, final String userdata,
+                                          final Set<String> tags) {
+            return builder()
+                    .hostname(hostname)
+                    .plan(plan)
+                    .billingCycle(billingCycle)
+                    .facility(facility)
+                    .features(features)
+                    .operatingSystem(operatingSystem)
+                    .locked(locked)
+                    .userdata(userdata)
+                    .tags(tags)
+                    .build();
+        }
+
+        public static Builder builder() {
+            return new AutoValue_Device_CreateDevice.Builder();
+        }
+
+        @AutoValue.Builder
+        public abstract static class Builder {
+
+            public abstract Builder hostname(String hostname);
+            public abstract Builder plan(String plan);
+            public abstract Builder billingCycle(String billingCycle);
+            public abstract Builder facility(String facility);
+            public abstract Builder features(Map<String, String> features);
+            public abstract Builder operatingSystem(String operatingSystem);
+            public abstract Builder locked(Boolean locked);
+            public abstract Builder userdata(String userdata);
+            public abstract Builder tags(Set<String> tags);
+
+           abstract Map<String, String> features();
+           abstract Set<String> tags();
+
+           abstract CreateDevice autoBuild();
+
+           public CreateDevice build() {
+              return tags(tags() != null ? ImmutableSet.copyOf(tags()) : ImmutableSet.<String> of())
+                      .features(features() != null ? ImmutableMap.copyOf(features()) : ImmutableMap.<String, String> of())
+                      .autoBuild();
+           }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/domain/ProvisioningEvent.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/ProvisioningEvent.java b/packet/src/main/java/org/jclouds/packet/domain/ProvisioningEvent.java
new file mode 100644
index 0000000..3df979f
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/domain/ProvisioningEvent.java
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.domain;
+
+import java.util.Date;
+import java.util.List;
+
+import org.jclouds.javax.annotation.Nullable;
+import org.jclouds.json.SerializedNames;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
+
+@AutoValue
+public abstract class ProvisioningEvent {
+
+   @Nullable
+   public abstract String id();
+   public abstract String type();
+   public abstract String body();
+   @Nullable
+   public abstract Date createdAt();
+   public abstract List<Href> relationships();
+   public abstract String interpolated();
+   @Nullable
+   public abstract String href();
+
+   @SerializedNames({"id", "type", "body", "created_at", "relationships", "interpolated", "href"})
+   public static ProvisioningEvent create(String id, String type, String body, Date createdAt,
+                                          List<Href> relationships, String interpolated, String href) {
+      return new AutoValue_ProvisioningEvent(id, type, body, createdAt,
+              relationships == null ? ImmutableList.<Href> of() : ImmutableList.copyOf(relationships),
+              interpolated,
+              href);
+   }
+
+   ProvisioningEvent() {}
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/features/DeviceApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/features/DeviceApi.java b/packet/src/main/java/org/jclouds/packet/features/DeviceApi.java
new file mode 100644
index 0000000..9f4c672
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/features/DeviceApi.java
@@ -0,0 +1,148 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.beans.ConstructorProperties;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+import org.jclouds.Fallbacks;
+import org.jclouds.Fallbacks.NullOnNotFoundOr404;
+import org.jclouds.Fallbacks.VoidOnNotFoundOr404;
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.collect.PagedIterable;
+import org.jclouds.collect.internal.Arg0ToPagedIterable;
+import org.jclouds.http.functions.ParseJson;
+import org.jclouds.javax.annotation.Nullable;
+import org.jclouds.json.Json;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.domain.ActionType;
+import org.jclouds.packet.domain.Device;
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.internal.PaginatedCollection;
+import org.jclouds.packet.domain.options.ListOptions;
+import org.jclouds.packet.filters.AddApiVersionToRequest;
+import org.jclouds.packet.filters.AddXAuthTokenToRequest;
+import org.jclouds.rest.annotations.BinderParam;
+import org.jclouds.rest.annotations.Fallback;
+import org.jclouds.rest.annotations.MapBinder;
+import org.jclouds.rest.annotations.PayloadParam;
+import org.jclouds.rest.annotations.RequestFilters;
+import org.jclouds.rest.annotations.ResponseParser;
+import org.jclouds.rest.annotations.Transform;
+import org.jclouds.rest.binders.BindToJsonPayload;
+
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+import com.google.inject.TypeLiteral;
+
+@Path("/projects/{projectId}/devices")
+@Consumes(MediaType.APPLICATION_JSON)
+@RequestFilters({AddXAuthTokenToRequest.class, AddApiVersionToRequest.class})
+public interface DeviceApi {
+
+   @Named("device:list")
+   @GET
+   @ResponseParser(ParseDevices.class)
+   @Transform(ParseDevices.ToPagedIterable.class)
+   @Fallback(Fallbacks.EmptyPagedIterableOnNotFoundOr404.class)
+   PagedIterable<Device> list();
+
+   @Named("device:list")
+   @GET
+   @ResponseParser(ParseDevices.class)
+   @Fallback(Fallbacks.EmptyIterableWithMarkerOnNotFoundOr404.class)
+   IterableWithMarker<Device> list(ListOptions options);
+
+   final class ParseDevices extends ParseJson<ParseDevices.Devices> {
+      @Inject
+      ParseDevices(Json json) {
+         super(json, TypeLiteral.get(Devices.class));
+      }
+
+       private static class Devices extends PaginatedCollection<Device> {
+         @ConstructorProperties({"devices", "meta"})
+         public Devices(List<Device> items, Meta meta) {
+            super(items, meta);
+         }
+      }
+
+      public static class ToPagedIterable extends Arg0ToPagedIterable.FromCaller<Device, ToPagedIterable> {
+
+         private final PacketApi api;
+         private final Function<Href, ListOptions> hrefToOptions;
+
+         @Inject
+         ToPagedIterable(PacketApi api, Function<Href, ListOptions> hrefToOptions) {
+            this.api = api;
+            this.hrefToOptions = hrefToOptions;
+         }
+
+         @Override
+         protected Function<Object, IterableWithMarker<Device>> markerToNextForArg0(Optional<Object> arg0) {
+            String projectId = arg0.get().toString();
+            final DeviceApi deviceApi = api.deviceApi(projectId);
+            return new Function<Object, IterableWithMarker<Device>>() {
+
+               @SuppressWarnings("unchecked")
+               @Override
+               public IterableWithMarker<Device> apply(Object input) {
+                  ListOptions listOptions = hrefToOptions.apply(Href.class.cast(input));
+                  return IterableWithMarker.class.cast(deviceApi.list(listOptions));
+               }
+
+            };
+         }
+      }
+   }
+
+   @Named("device:create")
+   @POST
+   @Produces(MediaType.APPLICATION_JSON)
+   Device create(@BinderParam(BindToJsonPayload.class) Device.CreateDevice device);
+
+
+   @Named("device:get")
+   @GET
+   @Path("/{id}")
+   @Fallback(NullOnNotFoundOr404.class)
+   @Nullable
+   Device get(@PathParam("id") String id);
+
+   @Named("device:delete")
+   @DELETE
+   @Path("/{id}")
+   @Fallback(VoidOnNotFoundOr404.class)
+   void delete(@PathParam("id") String id);
+
+   @Named("device:actions")
+   @POST
+   @Path("/{id}/actions")
+   @MapBinder(BindToJsonPayload.class)
+   void actions(@PathParam("id") String id, @PayloadParam("type") ActionType type);
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/features/FacilityApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/features/FacilityApi.java b/packet/src/main/java/org/jclouds/packet/features/FacilityApi.java
new file mode 100644
index 0000000..bde9898
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/features/FacilityApi.java
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.beans.ConstructorProperties;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.MediaType;
+
+import org.jclouds.Fallbacks;
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.collect.PagedIterable;
+import org.jclouds.http.functions.ParseJson;
+import org.jclouds.json.Json;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.domain.Facility;
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.internal.PaginatedCollection;
+import org.jclouds.packet.domain.options.ListOptions;
+import org.jclouds.packet.filters.AddApiVersionToRequest;
+import org.jclouds.packet.filters.AddXAuthTokenToRequest;
+import org.jclouds.packet.functions.BaseToPagedIterable;
+import org.jclouds.rest.annotations.Fallback;
+import org.jclouds.rest.annotations.RequestFilters;
+import org.jclouds.rest.annotations.ResponseParser;
+import org.jclouds.rest.annotations.Transform;
+
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+import com.google.inject.TypeLiteral;
+
+@Path("/facilities")
+@Consumes(MediaType.APPLICATION_JSON)
+@RequestFilters({ AddXAuthTokenToRequest.class, AddApiVersionToRequest.class} )
+public interface FacilityApi {
+
+    @Named("facility:list")
+    @GET
+    @ResponseParser(ParseFacilities.class)
+    @Transform(ParseFacilities.ToPagedIterable.class)
+    @Fallback(Fallbacks.EmptyPagedIterableOnNotFoundOr404.class)
+    PagedIterable<Facility> list();
+
+    @Named("facility:list")
+    @GET
+    @ResponseParser(ParseFacilities.class)
+    @Fallback(Fallbacks.EmptyIterableWithMarkerOnNotFoundOr404.class)
+    IterableWithMarker<Facility> list(ListOptions options);
+
+    final class ParseFacilities extends ParseJson<ParseFacilities.Facilities> {
+        @Inject
+        ParseFacilities(Json json) {
+            super(json, TypeLiteral.get(Facilities.class));
+        }
+
+        private static class Facilities extends PaginatedCollection<Facility> {
+            @ConstructorProperties({ "facilities", "meta" })
+            public Facilities(List<Facility> items, Meta meta) {
+                super(items, meta);
+            }
+        }
+
+        private static class ToPagedIterable extends BaseToPagedIterable<Facility, ListOptions> {
+            @Inject ToPagedIterable(PacketApi api, Function<Href, ListOptions> hrefToOptions) {
+                super(api, hrefToOptions);
+            }
+
+            @Override
+            protected IterableWithMarker<Facility> fetchPageUsingOptions(ListOptions options, Optional<Object> arg0) {
+                return api.facilityApi().list(options);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/features/OperatingSystemApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/features/OperatingSystemApi.java b/packet/src/main/java/org/jclouds/packet/features/OperatingSystemApi.java
new file mode 100644
index 0000000..401b1e9
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/features/OperatingSystemApi.java
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.beans.ConstructorProperties;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.MediaType;
+
+import org.jclouds.Fallbacks;
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.collect.PagedIterable;
+import org.jclouds.http.functions.ParseJson;
+import org.jclouds.json.Json;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.OperatingSystem;
+import org.jclouds.packet.domain.internal.PaginatedCollection;
+import org.jclouds.packet.domain.options.ListOptions;
+import org.jclouds.packet.filters.AddApiVersionToRequest;
+import org.jclouds.packet.filters.AddXAuthTokenToRequest;
+import org.jclouds.packet.functions.BaseToPagedIterable;
+import org.jclouds.rest.annotations.Fallback;
+import org.jclouds.rest.annotations.RequestFilters;
+import org.jclouds.rest.annotations.ResponseParser;
+import org.jclouds.rest.annotations.Transform;
+
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+import com.google.inject.TypeLiteral;
+
+@Path("/operating-systems")
+@Consumes(MediaType.APPLICATION_JSON)
+@RequestFilters({ AddXAuthTokenToRequest.class, AddApiVersionToRequest.class} )
+public interface OperatingSystemApi {
+
+    @Named("operatingsystem:list")
+    @GET
+    @ResponseParser(ParseOperatingSystems.class)
+    @Transform(ParseOperatingSystems.ToPagedIterable.class)
+    @Fallback(Fallbacks.EmptyPagedIterableOnNotFoundOr404.class)
+    PagedIterable<OperatingSystem> list();
+
+    @Named("operatingsystem:list")
+    @GET
+    @ResponseParser(ParseOperatingSystems.class)
+    @Fallback(Fallbacks.EmptyIterableWithMarkerOnNotFoundOr404.class)
+    IterableWithMarker<OperatingSystem> list(ListOptions options);
+
+    final class ParseOperatingSystems extends ParseJson<ParseOperatingSystems.OperatingSystems> {
+        @Inject
+        ParseOperatingSystems(Json json) {
+            super(json, TypeLiteral.get(ParseOperatingSystems.OperatingSystems.class));
+        }
+
+        private static class OperatingSystems extends PaginatedCollection<OperatingSystem> {
+            @ConstructorProperties({ "operating_systems", "meta" })
+            public OperatingSystems(List<OperatingSystem> items, Meta meta) {
+                super(items, meta);
+            }
+        }
+
+        private static class ToPagedIterable extends BaseToPagedIterable<OperatingSystem, ListOptions> {
+            @Inject ToPagedIterable(PacketApi api, Function<Href, ListOptions> hrefToOptions) {
+                super(api, hrefToOptions);
+            }
+
+            @Override
+            protected IterableWithMarker<OperatingSystem> fetchPageUsingOptions(ListOptions options, Optional<Object> arg0) {
+                return api.operatingSystemApi().list(options);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/features/PlanApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/features/PlanApi.java b/packet/src/main/java/org/jclouds/packet/features/PlanApi.java
new file mode 100644
index 0000000..7ed5c3a
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/features/PlanApi.java
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.beans.ConstructorProperties;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.MediaType;
+
+import org.jclouds.Fallbacks;
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.collect.PagedIterable;
+import org.jclouds.http.functions.ParseJson;
+import org.jclouds.json.Json;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.Plan;
+import org.jclouds.packet.domain.internal.PaginatedCollection;
+import org.jclouds.packet.domain.options.ListOptions;
+import org.jclouds.packet.filters.AddApiVersionToRequest;
+import org.jclouds.packet.filters.AddXAuthTokenToRequest;
+import org.jclouds.packet.functions.BaseToPagedIterable;
+import org.jclouds.rest.annotations.Fallback;
+import org.jclouds.rest.annotations.RequestFilters;
+import org.jclouds.rest.annotations.ResponseParser;
+import org.jclouds.rest.annotations.Transform;
+
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+import com.google.inject.TypeLiteral;
+
+@Path("/plans")
+@Consumes(MediaType.APPLICATION_JSON)
+@RequestFilters({ AddXAuthTokenToRequest.class, AddApiVersionToRequest.class} )
+public interface PlanApi {
+
+    @Named("plan:list")
+    @GET
+    @ResponseParser(ParsePlans.class)
+    @Transform(ParsePlans.ToPagedIterable.class)
+    @Fallback(Fallbacks.EmptyPagedIterableOnNotFoundOr404.class)
+    PagedIterable<Plan> list();
+
+    @Named("plan:list")
+    @GET
+    @ResponseParser(ParsePlans.class)
+    @Fallback(Fallbacks.EmptyIterableWithMarkerOnNotFoundOr404.class)
+    IterableWithMarker<Plan> list(ListOptions options);
+
+    final class ParsePlans extends ParseJson<ParsePlans.Plans> {
+        @Inject
+        ParsePlans(Json json) {
+            super(json, TypeLiteral.get(ParsePlans.Plans.class));
+        }
+
+        private static class Plans extends PaginatedCollection<Plan> {
+            @ConstructorProperties({ "plans", "meta" })
+            public Plans(List<Plan> items, Meta meta) {
+                super(items, meta);
+            }
+        }
+
+        private static class ToPagedIterable extends BaseToPagedIterable<Plan, ListOptions> {
+            @Inject ToPagedIterable(PacketApi api, Function<Href, ListOptions> hrefToOptions) {
+                super(api, hrefToOptions);
+            }
+
+            @Override
+            protected IterableWithMarker<Plan> fetchPageUsingOptions(ListOptions options, Optional<Object> arg0) {
+                return api.planApi().list(options);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java b/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
index afdf1ef..9da50d0 100644
--- a/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
+++ b/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
@@ -53,7 +53,6 @@ import com.google.inject.TypeLiteral;
 @RequestFilters({ AddXAuthTokenToRequest.class, AddApiVersionToRequest.class} )
 public interface ProjectApi {
 
-
     @Named("project:list")
     @GET
     @ResponseParser(ParseProjects.class)
@@ -81,8 +80,8 @@ public interface ProjectApi {
         }
 
         private static class ToPagedIterable extends BaseToPagedIterable<Project, ListOptions> {
-            @Inject ToPagedIterable(PacketApi api, Function<Href, ListOptions> linkToOptions) {
-                super(api, linkToOptions);
+            @Inject ToPagedIterable(PacketApi api, Function<Href, ListOptions> hrefToOptions) {
+                super(api, hrefToOptions);
             }
 
             @Override
@@ -91,4 +90,5 @@ public interface ProjectApi {
             }
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/features/SshKeyApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/features/SshKeyApi.java b/packet/src/main/java/org/jclouds/packet/features/SshKeyApi.java
new file mode 100644
index 0000000..cd22107
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/features/SshKeyApi.java
@@ -0,0 +1,122 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.beans.ConstructorProperties;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+import org.jclouds.Fallbacks;
+import org.jclouds.Fallbacks.NullOnNotFoundOr404;
+import org.jclouds.Fallbacks.VoidOnNotFoundOr404;
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.collect.PagedIterable;
+import org.jclouds.http.functions.ParseJson;
+import org.jclouds.javax.annotation.Nullable;
+import org.jclouds.json.Json;
+import org.jclouds.packet.PacketApi;
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.SshKey;
+import org.jclouds.packet.domain.internal.PaginatedCollection;
+import org.jclouds.packet.domain.options.ListOptions;
+import org.jclouds.packet.filters.AddApiVersionToRequest;
+import org.jclouds.packet.filters.AddXAuthTokenToRequest;
+import org.jclouds.packet.functions.BaseToPagedIterable;
+import org.jclouds.rest.annotations.Fallback;
+import org.jclouds.rest.annotations.MapBinder;
+import org.jclouds.rest.annotations.PayloadParam;
+import org.jclouds.rest.annotations.RequestFilters;
+import org.jclouds.rest.annotations.ResponseParser;
+import org.jclouds.rest.annotations.Transform;
+import org.jclouds.rest.binders.BindToJsonPayload;
+
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+import com.google.inject.TypeLiteral;
+
+@Path("/ssh-keys")
+@Consumes(MediaType.APPLICATION_JSON)
+@RequestFilters({ AddXAuthTokenToRequest.class, AddApiVersionToRequest.class} )
+public interface SshKeyApi {
+
+    @Named("sshkey:list")
+    @GET
+    @ResponseParser(ParseSshKeys.class)
+    @Transform(ParseSshKeys.ToPagedIterable.class)
+    @Fallback(Fallbacks.EmptyPagedIterableOnNotFoundOr404.class)
+    PagedIterable<SshKey> list();
+
+    @Named("sshkey:list")
+    @GET
+    @ResponseParser(ParseSshKeys.class)
+    @Fallback(Fallbacks.EmptyIterableWithMarkerOnNotFoundOr404.class)
+    IterableWithMarker<SshKey> list(ListOptions options);
+
+    final class ParseSshKeys extends ParseJson<ParseSshKeys.SshKeys> {
+        @Inject
+        ParseSshKeys(Json json) {
+            super(json, TypeLiteral.get(ParseSshKeys.SshKeys.class));
+        }
+
+        private static class SshKeys extends PaginatedCollection<SshKey> {
+            @ConstructorProperties({ "ssh_keys", "meta" })
+            public SshKeys(List<SshKey> items, Meta meta) {
+                super(items, meta);
+            }
+        }
+
+        private static class ToPagedIterable extends BaseToPagedIterable<SshKey, ListOptions> {
+            @Inject ToPagedIterable(PacketApi api, Function<Href, ListOptions> hrefToOptions) {
+                super(api, hrefToOptions);
+            }
+
+            @Override
+            protected IterableWithMarker<SshKey> fetchPageUsingOptions(ListOptions options, Optional<Object> arg0) {
+                return api.sshKeyApi().list(options);
+            }
+        }
+    }
+
+    @Named("sshkey:create")
+    @POST
+    @Produces(MediaType.APPLICATION_JSON)
+    @MapBinder(BindToJsonPayload.class)
+    SshKey create(@PayloadParam("label") String label, @PayloadParam("key") String key);
+
+    @Named("sshkey:get")
+    @GET
+    @Path("/{id}")
+    @Fallback(NullOnNotFoundOr404.class)
+    @Nullable
+    SshKey get(@PathParam("id") String id);
+
+    @Named("sshkey:delete")
+    @DELETE
+    @Path("/{id}")
+    @Fallback(VoidOnNotFoundOr404.class)
+    void delete(@PathParam("id") String id);
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/functions/BaseToPagedIterable.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/functions/BaseToPagedIterable.java b/packet/src/main/java/org/jclouds/packet/functions/BaseToPagedIterable.java
index c5c275b..abc59a2 100644
--- a/packet/src/main/java/org/jclouds/packet/functions/BaseToPagedIterable.java
+++ b/packet/src/main/java/org/jclouds/packet/functions/BaseToPagedIterable.java
@@ -35,12 +35,12 @@ import com.google.common.base.Optional;
  */
 public abstract class BaseToPagedIterable<T, O extends ListOptions> extends
         Arg0ToPagedIterable<T, BaseToPagedIterable<T, O>> {
-   private final Function<Href, O> linkToOptions;
+   private final Function<Href, O> hrefToOptions;
    protected final PacketApi api;
 
-   @Inject protected BaseToPagedIterable(PacketApi api, Function<Href, O> linkToOptions) {
+   @Inject protected BaseToPagedIterable(PacketApi api, Function<Href, O> hrefToOptions) {
       this.api = api;
-      this.linkToOptions = linkToOptions;
+      this.hrefToOptions = hrefToOptions;
    }
 
    protected abstract IterableWithMarker<T> fetchPageUsingOptions(O options, Optional<Object> arg0);
@@ -50,7 +50,7 @@ public abstract class BaseToPagedIterable<T, O extends ListOptions> extends
       return new Function<Object, IterableWithMarker<T>>() {
          @Override
          public IterableWithMarker<T> apply(Object input) {
-            O nextOptions = linkToOptions.apply(Href.class.cast(input));
+            O nextOptions = hrefToOptions.apply(Href.class.cast(input));
             return fetchPageUsingOptions(nextOptions, arg0);
          }
       };

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/functions/HrefToListOptions.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/functions/HrefToListOptions.java b/packet/src/main/java/org/jclouds/packet/functions/HrefToListOptions.java
new file mode 100644
index 0000000..d380b26
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/functions/HrefToListOptions.java
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.functions;
+
+import java.net.URI;
+
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.options.ListOptions;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Multimap;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Strings.emptyToNull;
+import static com.google.common.collect.Iterables.getFirst;
+import static org.jclouds.http.utils.Queries.queryParser;
+import static org.jclouds.packet.domain.options.ListOptions.PAGE_PARAM;
+import static org.jclouds.packet.domain.options.ListOptions.PER_PAGE_PARAM;
+
+/**
+ * Transforms an href returned by the API into a {@link ListOptions} that can be
+ * used to perform a request to get another page of a paginated list.
+ */
+public class HrefToListOptions implements Function<Href, ListOptions> {
+
+   @Override
+   public ListOptions apply(Href input) {
+      checkNotNull(input, "input cannot be null");
+
+      Multimap<String, String> queryParams = queryParser().apply(URI.create(input.href()).getQuery());
+      String nextPage = getFirstOrNull(PAGE_PARAM, queryParams);
+      String nextPerPage = getFirstOrNull(PER_PAGE_PARAM, queryParams);
+
+      ListOptions options = new ListOptions();
+      if (nextPage != null) {
+         options.page(Integer.parseInt(nextPage));
+      }
+      if (nextPerPage != null) {
+         options.perPage(Integer.parseInt(nextPerPage));
+      }
+
+      return options;
+   }
+
+   public static String getFirstOrNull(String key, Multimap<String, String> params) {
+      return params.containsKey(key) ? emptyToNull(getFirst(params.get(key), null)) : null;
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/main/java/org/jclouds/packet/functions/LinkToListOptions.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/functions/LinkToListOptions.java b/packet/src/main/java/org/jclouds/packet/functions/LinkToListOptions.java
deleted file mode 100644
index 4aef811..0000000
--- a/packet/src/main/java/org/jclouds/packet/functions/LinkToListOptions.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.
- */
-package org.jclouds.packet.functions;
-
-import java.net.URI;
-
-import org.jclouds.packet.domain.Href;
-import org.jclouds.packet.domain.options.ListOptions;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Multimap;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Strings.emptyToNull;
-import static com.google.common.collect.Iterables.getFirst;
-import static org.jclouds.http.utils.Queries.queryParser;
-import static org.jclouds.packet.domain.options.ListOptions.PAGE_PARAM;
-import static org.jclouds.packet.domain.options.ListOptions.PER_PAGE_PARAM;
-
-/**
- * Transforms an href returned by the API into a {@link ListOptions} that can be
- * used to perform a request to get another page of a paginated list.
- */
-public class LinkToListOptions implements Function<Href, ListOptions> {
-
-   @Override
-   public ListOptions apply(Href input) {
-      checkNotNull(input, "input cannot be null");
-
-      Multimap<String, String> queryParams = queryParser().apply(URI.create(input.href()).getQuery());
-      String nextPage = getFirstOrNull(PAGE_PARAM, queryParams);
-      String nextPerPage = getFirstOrNull(PER_PAGE_PARAM, queryParams);
-
-      ListOptions options = new ListOptions();
-      if (nextPage != null) {
-         options.page(Integer.parseInt(nextPage));
-      }
-      if (nextPerPage != null) {
-         options.perPage(Integer.parseInt(nextPerPage));
-      }
-
-      return options;
-   }
-
-   public static String getFirstOrNull(String key, Multimap<String, String> params) {
-      return params.containsKey(key) ? emptyToNull(getFirst(params.get(key), null)) : null;
-   }
-
-}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/DeviceApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/DeviceApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/features/DeviceApiLiveTest.java
new file mode 100644
index 0000000..36c08f1
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/DeviceApiLiveTest.java
@@ -0,0 +1,138 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jclouds.packet.compute.internal.BasePacketApiLiveTest;
+import org.jclouds.packet.domain.ActionType;
+import org.jclouds.packet.domain.BillingCycle;
+import org.jclouds.packet.domain.Device;
+import org.jclouds.packet.domain.SshKey;
+import org.jclouds.ssh.SshKeys;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+import static org.testng.util.Strings.isNullOrEmpty;
+
+@Test(groups = "live", testName = "DeviceApiLiveTest")
+public class DeviceApiLiveTest extends BasePacketApiLiveTest {
+
+   private SshKey sshKey;
+   private String deviceId;
+
+   @BeforeClass
+   public void setupDevice() {
+      Map<String, String> keyPair = SshKeys.generate();
+      sshKey = api.sshKeyApi().create(prefix + "-device-livetest", keyPair.get("public"));
+   }
+
+   @AfterClass(alwaysRun = true)
+   public void tearDown() {
+      if (sshKey != null) {
+         api.sshKeyApi().delete(sshKey.id());
+      }
+   }
+
+   public void testCreate() {
+      Device deviceCreated = api().create(
+              Device.CreateDevice.builder()
+                      .hostname(prefix + "-device-livetest")
+                      .plan("baremetal_0")
+                      .billingCycle(BillingCycle.HOURLY.value())
+                      .facility("ewr1")
+                      .features(ImmutableMap.<String, String>of())
+                      .operatingSystem("ubuntu_16_04")
+                      .locked(false)
+                      .userdata("")
+                      .tags(ImmutableSet.<String> of())
+                      .build()
+      );
+      deviceId = deviceCreated.id();
+      assertNodeRunning(deviceId);
+      Device device = api().get(deviceId);
+      assertNotNull(device, "Device must not be null");
+   }
+
+   @Test(groups = "live", dependsOnMethods = "testCreate")
+   public void testReboot() {
+      api().actions(deviceId, ActionType.REBOOT);
+      assertNodeRunning(deviceId);
+   }
+
+   @Test(groups = "live", dependsOnMethods = "testReboot")
+   public void testPowerOff() {
+      api().actions(deviceId, ActionType.POWER_OFF);
+      assertNodeTerminated(deviceId);
+   }
+
+   @Test(groups = "live", dependsOnMethods = "testPowerOff")
+   public void testPowerOn() {
+      api().actions(deviceId, ActionType.POWER_ON);
+      assertNodeRunning(deviceId);
+   }
+   
+   @Test(dependsOnMethods = "testCreate")
+   public void testList() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(Iterables.all(api().list().concat(), new Predicate<Device>() {
+         @Override
+         public boolean apply(Device input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All devices must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some devices to be returned");
+   }
+
+   @Test(dependsOnMethods = "testCreate")
+   public void testListOnePage() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(api().list(page(1).perPage(5)).allMatch(new Predicate<Device>() {
+         @Override
+         public boolean apply(Device input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All devices must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some devices to be returned");
+   }
+
+   @Test(dependsOnMethods = "testList", alwaysRun = true)
+   public void testDelete() throws InterruptedException {
+      if (deviceId != null) {
+         api().delete(deviceId);
+         assertNodeTerminated(deviceId);
+         assertNull(api().get(deviceId));
+      }
+   }
+
+   private DeviceApi api() {
+      return api.deviceApi(identity);
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/DeviceApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/DeviceApiMockTest.java b/packet/src/test/java/org/jclouds/packet/features/DeviceApiMockTest.java
new file mode 100644
index 0000000..705955c
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/DeviceApiMockTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import org.jclouds.packet.compute.internal.BasePacketApiMockTest;
+import org.jclouds.packet.domain.ActionType;
+import org.jclouds.packet.domain.BillingCycle;
+import org.jclouds.packet.domain.Device;
+import org.jclouds.packet.domain.Device.CreateDevice;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+import static com.google.common.collect.Iterables.isEmpty;
+import static com.google.common.collect.Iterables.size;
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+@Test(groups = "unit", testName = "DeviceApiMockTest", singleThreaded = true)
+public class DeviceApiMockTest extends BasePacketApiMockTest {
+
+   public void testListDevices() throws InterruptedException {
+      server.enqueue(jsonResponse("/devices-first.json"));
+      server.enqueue(jsonResponse("/devices-last.json"));
+
+      Iterable<Device> devices = api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").list().concat();
+
+      assertEquals(size(devices), 7); // Force the PagedIterable to advance
+      assertEquals(server.getRequestCount(), 2);
+
+      assertSent(server, "GET", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices");
+      assertSent(server, "GET", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=2");
+   }
+
+   public void testListDevicesReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<Device> devices = api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").list().concat();
+
+      assertTrue(isEmpty(devices));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices");
+   }
+
+   public void testListDevicesWithOptions() throws InterruptedException {
+      server.enqueue(jsonResponse("/devices-first.json"));
+
+      Iterable<Device> devices = api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").list(page(1).perPage(5));
+
+      assertEquals(size(devices), 5);
+      assertEquals(server.getRequestCount(), 1);
+
+      assertSent(server, "GET", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=1&per_page=5");
+   }
+
+   public void testListDevicesWithOptionsReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<Device> actions = api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").list(page(1).perPage(5));
+
+      assertTrue(isEmpty(actions));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=1&per_page=5");
+   }
+
+   public void testGetDevice() throws InterruptedException {
+      server.enqueue(jsonResponse("/device.json"));
+
+      Device device = api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").get("1");
+
+      assertEquals(device, objectFromResource("/device.json", Device.class));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/1");
+   }
+
+   public void testGetDeviceReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Device device = api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").get("1");
+
+      assertNull(device);
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/1");
+   }
+
+   public void testCreateDevice() throws InterruptedException {
+      server.enqueue(jsonResponse("/device-create-res.json"));
+
+      Device device = api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").create(
+              CreateDevice.builder()
+                      .hostname("jclouds-device-livetest")
+                      .plan("baremetal_0")
+                      .billingCycle(BillingCycle.HOURLY.value())
+                      .facility("ewr1")
+                      .features(ImmutableMap.<String, String>of())
+                      .operatingSystem("ubuntu_16_04")
+                      .locked(false)
+                      .userdata("")
+                      .tags(ImmutableSet.<String> of())
+                      .build()
+      );
+
+      assertEquals(device, objectFromResource("/device-create-res.json", Device.class));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "POST", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices", stringFromResource("/device-create-req.json"));
+   }
+
+   public void testDeleteDevice() throws InterruptedException {
+      server.enqueue(response204());
+
+      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").delete("1");
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "DELETE", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/1");
+   }
+
+   public void testDeleteDeviceReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").delete("1");
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "DELETE", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/1");
+   }
+
+   public void testActionPowerOn() throws InterruptedException {
+      server.enqueue(jsonResponse("/power-on.json"));
+
+      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").actions("deviceId", ActionType.POWER_ON);
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "POST", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/deviceId/actions");
+   }
+
+   public void testActionPowerOff() throws InterruptedException {
+      server.enqueue(jsonResponse("/power-off.json"));
+
+      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").actions("deviceId", ActionType.POWER_OFF);
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "POST", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/deviceId/actions");
+   }
+
+   public void testActionReboot() throws InterruptedException {
+      server.enqueue(jsonResponse("/reboot.json"));
+
+      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").actions("deviceId", ActionType.REBOOT);
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "POST", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/deviceId/actions");
+   }
+
+   public void testActionRescue() throws InterruptedException {
+      server.enqueue(jsonResponse("/rescue.json"));
+
+      api.deviceApi("93907f48-adfe-43ed-ad89-0e6e83721a54").actions("deviceId", ActionType.RESCUE);
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "POST", "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices/deviceId/actions");
+   }
+   
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/FacilityApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/FacilityApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/features/FacilityApiLiveTest.java
new file mode 100644
index 0000000..95fc857
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/FacilityApiLiveTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jclouds.packet.compute.internal.BasePacketApiMockTest;
+import org.jclouds.packet.domain.Facility;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertTrue;
+import static org.testng.util.Strings.isNullOrEmpty;
+
+@Test(groups = "live", testName = "FacilityApiLiveTest")
+public class FacilityApiLiveTest extends BasePacketApiMockTest {
+
+   public void testList() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(Iterables.all(api().list().concat(), new Predicate<Facility>() {
+         @Override
+         public boolean apply(Facility input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All facilities must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some facilities to be returned");
+   }
+
+   public void testListOnePage() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(api().list(page(1).perPage(5)).allMatch(new Predicate<Facility>() {
+         @Override
+         public boolean apply(Facility input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All facilities must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some facilities to be returned");
+   }
+
+   private FacilityApi api() {
+      return api.facilityApi();
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/FacilityApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/FacilityApiMockTest.java b/packet/src/test/java/org/jclouds/packet/features/FacilityApiMockTest.java
new file mode 100644
index 0000000..764fa7b
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/FacilityApiMockTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import org.jclouds.packet.compute.internal.BasePacketApiMockTest;
+import org.jclouds.packet.domain.Facility;
+import org.testng.annotations.Test;
+
+import static com.google.common.collect.Iterables.isEmpty;
+import static com.google.common.collect.Iterables.size;
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+@Test(groups = "unit", testName = "FacilityApiMockTest", singleThreaded = true)
+public class FacilityApiMockTest extends BasePacketApiMockTest {
+
+   public void testListFacilities() throws InterruptedException {
+      server.enqueue(jsonResponse("/facilities-first.json"));
+      server.enqueue(jsonResponse("/facilities-last.json"));
+
+      Iterable<Facility> facilities = api.facilityApi().list().concat();
+
+      assertEquals(size(facilities), 3); // Force the PagedIterable to advance
+      assertEquals(server.getRequestCount(), 2);
+
+      assertSent(server, "GET", "/facilities");
+      assertSent(server, "GET", "/facilities?page=2");
+   }
+
+   public void testListFacilitiesReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<Facility> facilities = api.facilityApi().list().concat();
+
+      assertTrue(isEmpty(facilities));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/facilities");
+   }
+
+   public void testListFacilitiesWithOptions() throws InterruptedException {
+      server.enqueue(jsonResponse("/facilities-first.json"));
+
+      Iterable<Facility> actions = api.facilityApi().list(page(1).perPage(2));
+
+      assertEquals(size(actions), 2);
+      assertEquals(server.getRequestCount(), 1);
+
+      assertSent(server, "GET", "/facilities?page=1&per_page=2");
+   }
+
+   public void testListFacilitiesWithOptionsReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<Facility> actions = api.facilityApi().list(page(1).perPage(2));
+
+      assertTrue(isEmpty(actions));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/facilities?page=1&per_page=2");
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/OperatingSystemApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/OperatingSystemApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/features/OperatingSystemApiLiveTest.java
new file mode 100644
index 0000000..fd96a7e
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/OperatingSystemApiLiveTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jclouds.packet.compute.internal.BasePacketApiLiveTest;
+import org.jclouds.packet.domain.OperatingSystem;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertTrue;
+import static org.testng.util.Strings.isNullOrEmpty;
+
+@Test(groups = "live", testName = "OperatingSystemApiLiveTest")
+public class OperatingSystemApiLiveTest extends BasePacketApiLiveTest {
+
+   public void testList() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(Iterables.all(api().list().concat(), new Predicate<OperatingSystem>() {
+         @Override
+         public boolean apply(OperatingSystem input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All operating systems must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some operating systems to be returned");
+   }
+
+   public void testListOnePage() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(api().list(page(1).perPage(5)).allMatch(new Predicate<OperatingSystem>() {
+         @Override
+         public boolean apply(OperatingSystem input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All operating systems must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some operating systems to be returned");
+   }
+
+   private OperatingSystemApi api() {
+      return api.operatingSystemApi();
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/OperatingSystemApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/OperatingSystemApiMockTest.java b/packet/src/test/java/org/jclouds/packet/features/OperatingSystemApiMockTest.java
new file mode 100644
index 0000000..c0c332b
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/OperatingSystemApiMockTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import org.jclouds.packet.compute.internal.BasePacketApiMockTest;
+import org.jclouds.packet.domain.OperatingSystem;
+import org.testng.annotations.Test;
+
+import static com.google.common.collect.Iterables.isEmpty;
+import static com.google.common.collect.Iterables.size;
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+@Test(groups = "unit", testName = "OperatingSystemApiMockTest", singleThreaded = true)
+public class OperatingSystemApiMockTest extends BasePacketApiMockTest {
+
+   public void testListOperatingSystems() throws InterruptedException {
+
+      server.enqueue(jsonResponse("/operatingSystems-first.json"));
+      server.enqueue(jsonResponse("/operatingSystems-last.json"));
+
+      Iterable<OperatingSystem> operatingSystems = api.operatingSystemApi().list().concat();
+      assertEquals(size(operatingSystems), 14); // Force the PagedIterable to advance
+      assertEquals(server.getRequestCount(), 2);
+
+      assertSent(server, "GET", "/operating-systems");
+      assertSent(server, "GET", "/operating-systems?page=2");
+   }
+
+   public void testListOperatingSystemsReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<OperatingSystem> operatingSystems = api.operatingSystemApi().list().concat();
+
+      assertTrue(isEmpty(operatingSystems));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/operating-systems");
+   }
+
+   public void testListOperatingSystemsWithOptions() throws InterruptedException {
+      server.enqueue(jsonResponse("/operatingSystems-first.json"));
+
+      Iterable<OperatingSystem> operatingSystems = api.operatingSystemApi().list(page(1).perPage(5));
+
+      assertEquals(size(operatingSystems), 7);
+      assertEquals(server.getRequestCount(), 1);
+
+      assertSent(server, "GET", "/operating-systems?page=1&per_page=5");
+   }
+
+   public void testListOperatingSystemsWithOptionsReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<OperatingSystem> operatingSystems = api.operatingSystemApi().list(page(1).perPage(5));
+
+      assertTrue(isEmpty(operatingSystems));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/operating-systems?page=1&per_page=5");
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/PlanApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/PlanApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/features/PlanApiLiveTest.java
new file mode 100644
index 0000000..1b66e04
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/PlanApiLiveTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jclouds.packet.compute.internal.BasePacketApiLiveTest;
+import org.jclouds.packet.domain.Plan;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertTrue;
+import static org.testng.util.Strings.isNullOrEmpty;
+
+@Test(groups = "live", testName = "PlanApiLiveTest")
+public class PlanApiLiveTest extends BasePacketApiLiveTest {
+
+   public void testList() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(Iterables.all(api().list().concat(), new Predicate<Plan>() {
+         @Override
+         public boolean apply(Plan input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All plans must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some plans to be returned");
+   }
+
+   public void testListOnePage() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(api().list(page(1).perPage(5)).allMatch(new Predicate<Plan>() {
+         @Override
+         public boolean apply(Plan input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All plans must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some plans to be returned");
+   }
+
+   private PlanApi api() {
+      return api.planApi();
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/PlanApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/PlanApiMockTest.java b/packet/src/test/java/org/jclouds/packet/features/PlanApiMockTest.java
new file mode 100644
index 0000000..82683c8
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/PlanApiMockTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import org.jclouds.packet.compute.internal.BasePacketApiMockTest;
+import org.jclouds.packet.domain.Plan;
+import org.testng.annotations.Test;
+
+import static com.google.common.collect.Iterables.isEmpty;
+import static com.google.common.collect.Iterables.size;
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+@Test(groups = "unit", testName = "PlanApiMockTest", singleThreaded = true)
+public class PlanApiMockTest extends BasePacketApiMockTest {
+
+   public void testListPlans() throws InterruptedException {
+      server.enqueue(jsonResponse("/plans-first.json"));
+      server.enqueue(jsonResponse("/plans-last.json"));
+
+      Iterable<Plan> plans = api.planApi().list().concat();
+
+      assertEquals(size(plans), 7); // Force the PagedIterable to advance
+      assertEquals(server.getRequestCount(), 2);
+
+      assertSent(server, "GET", "/plans");
+      assertSent(server, "GET", "/plans?page=2");
+   }
+
+   public void testListPlansReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<Plan> plans = api.planApi().list().concat();
+
+      assertTrue(isEmpty(plans));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/plans");
+   }
+
+   public void testListPlansWithOptions() throws InterruptedException {
+      server.enqueue(jsonResponse("/plans-first.json"));
+
+      Iterable<Plan> plans = api.planApi().list(page(1).perPage(5));
+
+      assertEquals(size(plans), 4);
+      assertEquals(server.getRequestCount(), 1);
+
+      assertSent(server, "GET", "/plans?page=1&per_page=5");
+   }
+
+   public void testListPlansWithOptionsReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<Plan> plans = api.planApi().list(page(1).perPage(5));
+
+      assertTrue(isEmpty(plans));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/plans?page=1&per_page=5");
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/ProjectApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/ProjectApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/features/ProjectApiLiveTest.java
index 133e5ef..65fba8f 100644
--- a/packet/src/test/java/org/jclouds/packet/features/ProjectApiLiveTest.java
+++ b/packet/src/test/java/org/jclouds/packet/features/ProjectApiLiveTest.java
@@ -32,7 +32,7 @@ import static org.testng.util.Strings.isNullOrEmpty;
 @Test(groups = "live", testName = "ProjectApiLiveTest")
 public class ProjectApiLiveTest extends BasePacketApiLiveTest {
 
-   public void testListProjects() {
+   public void testList() {
       final AtomicInteger found = new AtomicInteger(0);
       assertTrue(Iterables.all(api().list().concat(), new Predicate<Project>() {
          @Override
@@ -43,8 +43,8 @@ public class ProjectApiLiveTest extends BasePacketApiLiveTest {
       }), "All projects must have the 'id' field populated");
       assertTrue(found.get() > 0, "Expected some projects to be returned");
    }
-   
-   public void testListActionsOnePage() {
+
+   public void testListOnePage() {
       final AtomicInteger found = new AtomicInteger(0);
       assertTrue(api().list(page(1).perPage(5)).allMatch(new Predicate<Project>() {
          @Override
@@ -55,7 +55,6 @@ public class ProjectApiLiveTest extends BasePacketApiLiveTest {
       }), "All projects must have the 'id' field populated");
       assertTrue(found.get() > 0, "Expected some projects to be returned");
    }
-   
 
    private ProjectApi api() {
       return api.projectApi();

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/ProjectApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/ProjectApiMockTest.java b/packet/src/test/java/org/jclouds/packet/features/ProjectApiMockTest.java
index 2899020..d972395 100644
--- a/packet/src/test/java/org/jclouds/packet/features/ProjectApiMockTest.java
+++ b/packet/src/test/java/org/jclouds/packet/features/ProjectApiMockTest.java
@@ -74,6 +74,5 @@ public class ProjectApiMockTest extends BasePacketApiMockTest {
       assertEquals(server.getRequestCount(), 1);
       assertSent(server, "GET", "/projects?page=1&per_page=5");
    }
-   
 
 }


[2/9] jclouds-labs git commit: add compute functions

Posted by na...@apache.org.
add compute functions

- fix Device and PaginatedCollection
- improve DeviceToNodeMetadata


Project: http://git-wip-us.apache.org/repos/asf/jclouds-labs/repo
Commit: http://git-wip-us.apache.org/repos/asf/jclouds-labs/commit/cbe306f5
Tree: http://git-wip-us.apache.org/repos/asf/jclouds-labs/tree/cbe306f5
Diff: http://git-wip-us.apache.org/repos/asf/jclouds-labs/diff/cbe306f5

Branch: refs/heads/2.0.x
Commit: cbe306f53d566d7d7f124afa0e1aef1e75ee52ee
Parents: a812591
Author: Andrea Turli <an...@gmail.com>
Authored: Tue Jan 24 22:33:48 2017 +0100
Committer: Ignasi Barrera <na...@apache.org>
Committed: Thu Feb 2 15:23:25 2017 +0100

----------------------------------------------------------------------
 .../compute/functions/DeviceStateToStatus.java  |  45 ++++++++
 .../compute/functions/DeviceToNodeMetadata.java | 110 +++++++++++++++++++
 .../compute/functions/FacilityToLocation.java   |  55 ++++++++++
 .../functions/OperatingSystemToImage.java       |  54 +++++++++
 .../compute/functions/PlanToHardware.java       |  86 +++++++++++++++
 .../java/org/jclouds/packet/domain/Device.java  |   2 +-
 .../domain/internal/PaginatedCollection.java    |  10 +-
 7 files changed, 356 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cbe306f5/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceStateToStatus.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceStateToStatus.java b/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceStateToStatus.java
new file mode 100644
index 0000000..acb1f00
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceStateToStatus.java
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.functions;
+
+import javax.inject.Singleton;
+
+import org.jclouds.compute.domain.NodeMetadata.Status;
+import org.jclouds.packet.domain.Device;
+
+import com.google.common.base.Function;
+import com.google.common.base.Functions;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * Transforms an {@link Device.State} to the jclouds portable model.
+ */
+@Singleton
+public class DeviceStateToStatus implements Function<Device.State, Status> {
+
+   private static final Function<Device.State, Status> toPortableStatus = Functions.forMap(
+         ImmutableMap.<Device.State, Status> builder()
+               .put(Device.State.PROVISIONING, Status.PENDING)
+               .put(Device.State.ACTIVE, Status.RUNNING)
+               .build(),
+         Status.UNRECOGNIZED);
+
+   @Override
+   public Status apply(final Device.State input) {
+      return toPortableStatus.apply(input);
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cbe306f5/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceToNodeMetadata.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceToNodeMetadata.java b/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceToNodeMetadata.java
new file mode 100644
index 0000000..74d9d95
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/compute/functions/DeviceToNodeMetadata.java
@@ -0,0 +1,110 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.functions;
+
+import java.util.List;
+
+import javax.annotation.Resource;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.inject.Singleton;
+
+import org.jclouds.compute.domain.NodeMetadata;
+import org.jclouds.compute.domain.NodeMetadataBuilder;
+import org.jclouds.compute.functions.GroupNamingConvention;
+import org.jclouds.compute.reference.ComputeServiceConstants;
+import org.jclouds.logging.Logger;
+import org.jclouds.packet.domain.Device;
+import org.jclouds.packet.domain.IpAddress;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+
+import static com.google.common.collect.FluentIterable.from;
+
+/**
+ * Transforms an {@link Device} to the jclouds portable model.
+ */
+@Singleton
+public class DeviceToNodeMetadata implements Function<Device, NodeMetadata> {
+
+    @Resource
+    @Named(ComputeServiceConstants.COMPUTE_LOGGER)
+    protected Logger logger = Logger.NULL;
+
+    private final PlanToHardware planToHardware;
+    private final OperatingSystemToImage operatingSystemToImage;
+    private final FacilityToLocation facilityToLocation;
+    private final Function<Device.State, NodeMetadata.Status> toPortableStatus;
+    private final GroupNamingConvention groupNamingConvention;
+
+    @Inject
+    DeviceToNodeMetadata(PlanToHardware planToHardware, OperatingSystemToImage operatingSystemToImage, FacilityToLocation facilityToLocation,
+                         Function<Device.State, NodeMetadata.Status> toPortableStatus,
+                         GroupNamingConvention.Factory groupNamingConvention) {
+        this.planToHardware = planToHardware;
+        this.operatingSystemToImage = operatingSystemToImage;
+        this.facilityToLocation = facilityToLocation;
+        this.toPortableStatus = toPortableStatus;
+        this.groupNamingConvention = groupNamingConvention.createWithoutPrefix();
+    }
+
+   @Override
+   public NodeMetadata apply(Device input) {
+      return new NodeMetadataBuilder()
+              .ids(input.id())
+              .name(input.hostname())
+              .hostname(input.hostname())
+              .group(groupNamingConvention.extractGroup(input.hostname()))
+              .location(facilityToLocation.apply(input.facility()))
+              .hardware(planToHardware.apply(input.plan()))
+              .imageId(input.operatingSystem().slug())
+              .operatingSystem(operatingSystemToImage.apply(input.operatingSystem()).getOperatingSystem())
+              .status(toPortableStatus.apply(input.state()))
+              .publicAddresses(getPublicIpAddresses(input.ipAddresses()))
+              .privateAddresses(getPrivateIpAddresses(input.ipAddresses()))
+              .tags(input.tags())
+              .build();
+   }
+
+   private Iterable<String> getPublicIpAddresses(List<IpAddress> input) {
+      return filterAndTransformIpAddresses(input, new IsPublicIpAddress());
+   }
+
+   private Iterable<String> getPrivateIpAddresses(List<IpAddress> input) {
+      return filterAndTransformIpAddresses(input, Predicates.not(new IsPublicIpAddress()));
+   }
+
+   private Iterable<String> filterAndTransformIpAddresses(List<IpAddress> input, Predicate<IpAddress> filter) {
+      return from(input).filter(filter).transform(new IpAddressToIp());
+   }
+
+   private static class IpAddressToIp implements Function<IpAddress, String> {
+        @Override
+        public String apply(final IpAddress input) {
+            return input.address();
+        }
+    }
+
+    private static class IsPublicIpAddress implements Predicate<IpAddress> {
+        @Override
+        public boolean apply(IpAddress input) {
+            return input.publicAddress();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cbe306f5/packet/src/main/java/org/jclouds/packet/compute/functions/FacilityToLocation.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/functions/FacilityToLocation.java b/packet/src/main/java/org/jclouds/packet/compute/functions/FacilityToLocation.java
new file mode 100644
index 0000000..c8df1bc
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/compute/functions/FacilityToLocation.java
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.functions;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+
+import org.jclouds.domain.Location;
+import org.jclouds.domain.LocationBuilder;
+import org.jclouds.domain.LocationScope;
+import org.jclouds.location.suppliers.all.JustProvider;
+import org.jclouds.packet.domain.Facility;
+
+import com.google.common.base.Function;
+
+import static com.google.common.collect.Iterables.getOnlyElement;
+
+/**
+ * Transforms an {@link Facility} to the jclouds portable model.
+ */
+@Singleton
+public class FacilityToLocation implements Function<Facility, Location> {
+
+    private final JustProvider justProvider;
+
+    // allow us to lazy discover the provider of a resource
+    @Inject
+    FacilityToLocation(JustProvider justProvider) {
+        this.justProvider = justProvider;
+    }
+
+    @Override
+    public Location apply(final Facility facility) {
+        final LocationBuilder builder = new LocationBuilder();
+        builder.id(facility.code());
+        builder.description(facility.name());
+        builder.parent(getOnlyElement(justProvider.get()));
+        builder.scope(LocationScope.REGION);
+        return builder.build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cbe306f5/packet/src/main/java/org/jclouds/packet/compute/functions/OperatingSystemToImage.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/functions/OperatingSystemToImage.java b/packet/src/main/java/org/jclouds/packet/compute/functions/OperatingSystemToImage.java
new file mode 100644
index 0000000..d28b579
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/compute/functions/OperatingSystemToImage.java
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.functions;
+
+import javax.inject.Singleton;
+
+import org.jclouds.compute.domain.Image;
+import org.jclouds.compute.domain.ImageBuilder;
+import org.jclouds.packet.domain.Distribution;
+import org.jclouds.packet.domain.OperatingSystem;
+
+import com.google.common.base.Function;
+
+import static org.jclouds.compute.domain.OperatingSystem.builder;
+
+/**
+ * Transforms an {@link OperatingSystem} to the jclouds portable model.
+ */
+@Singleton
+public class OperatingSystemToImage implements Function<OperatingSystem, Image> {
+
+    @Override
+    public Image apply(final OperatingSystem input) {
+        ImageBuilder builder = new ImageBuilder();
+        builder.ids(input.slug());
+        builder.name(input.name());
+        builder.description(input.name());
+        builder.status(Image.Status.AVAILABLE);
+
+        builder.operatingSystem(builder()
+                .name(input.name())
+                .family(Distribution.fromValue(input.distribution()).osFamily())
+                .description(input.name())
+                .version(input.version())
+                .is64Bit(true)
+                .build());
+
+        return builder.build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cbe306f5/packet/src/main/java/org/jclouds/packet/compute/functions/PlanToHardware.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/compute/functions/PlanToHardware.java b/packet/src/main/java/org/jclouds/packet/compute/functions/PlanToHardware.java
new file mode 100644
index 0000000..e6ac2ef
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/compute/functions/PlanToHardware.java
@@ -0,0 +1,86 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.compute.functions;
+
+import javax.inject.Singleton;
+
+import org.jclouds.compute.domain.Hardware;
+import org.jclouds.compute.domain.HardwareBuilder;
+import org.jclouds.compute.domain.Processor;
+import org.jclouds.compute.domain.Volume;
+import org.jclouds.compute.domain.internal.VolumeImpl;
+import org.jclouds.packet.domain.Plan;
+import org.jclouds.packet.domain.Specs;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+/**
+ * Transforms an {@link Plan} to the jclouds portable model.
+ */
+@Singleton
+public class PlanToHardware implements Function<Plan, Hardware> {
+
+    @Override
+    public Hardware apply(Plan plan) {
+        HardwareBuilder builder = new HardwareBuilder()
+                .ids(plan.slug())
+                .name(plan.name())
+                .hypervisor("none")
+                .processors(getProcessors(plan))
+                .ram(getMemory(plan))
+                .volumes(getVolumes(plan));
+        return builder.build();
+    }
+
+    private Integer getMemory(Plan plan) {
+        if (plan.specs() == null || plan.specs().drives() == null) return 0;
+        String total = plan.specs().memory().total();
+        if (total.endsWith("GB")) {
+            return Integer.valueOf(total.substring(0, total.length() - 2)) * 1024;
+        } else {
+            throw new IllegalArgumentException("Cannot parse memory: " + plan.specs().memory());
+        }
+    }
+
+    private Iterable<Volume> getVolumes(Plan plan) {
+        if (plan.specs() == null || plan.specs().drives() == null) return Lists.newArrayList();
+
+        return Iterables.transform(plan.specs().drives(), new Function<Specs.Drive, Volume>() {
+            @Override
+            public Volume apply(Specs.Drive drive) {
+                return new VolumeImpl(
+                        drive.type(),
+                        Volume.Type.LOCAL,
+                        Float.parseFloat(drive.size().substring(0, drive.size().length() - 2)), null, true, false);
+            }
+        });
+    }
+
+
+    private Iterable<Processor> getProcessors(Plan plan) {
+        if (plan.specs() == null || plan.specs().cpus() == null) return Lists.newArrayList();
+        return Iterables.transform(plan.specs().cpus(), new Function<Specs.CPU, Processor>() {
+            @Override
+            public Processor apply(Specs.CPU input) {
+                // No cpu speed from Packet API, so assume more cores == faster
+                return new Processor(input.count(), input.count()); 
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cbe306f5/packet/src/main/java/org/jclouds/packet/domain/Device.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/Device.java b/packet/src/main/java/org/jclouds/packet/domain/Device.java
index 96e0b53..951d938 100644
--- a/packet/src/main/java/org/jclouds/packet/domain/Device.java
+++ b/packet/src/main/java/org/jclouds/packet/domain/Device.java
@@ -70,7 +70,7 @@ public abstract class Device {
     public abstract List<IpAddress> ipAddresses();
     public abstract List<ProvisioningEvent> provisioningEvents();
     public abstract Plan plan();
-    public abstract String rootPassword();
+    @Nullable public abstract String rootPassword();
     public abstract String userdata();
     public abstract String href();
 

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cbe306f5/packet/src/main/java/org/jclouds/packet/domain/internal/PaginatedCollection.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/domain/internal/PaginatedCollection.java b/packet/src/main/java/org/jclouds/packet/domain/internal/PaginatedCollection.java
index 047151e..14621bc 100644
--- a/packet/src/main/java/org/jclouds/packet/domain/internal/PaginatedCollection.java
+++ b/packet/src/main/java/org/jclouds/packet/domain/internal/PaginatedCollection.java
@@ -16,8 +16,6 @@
  */
 package org.jclouds.packet.domain.internal;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-
 import java.util.Iterator;
 import java.util.List;
 
@@ -30,6 +28,8 @@ import com.google.auto.value.AutoValue;
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * Base class for all collections that return paginated results.
  */
@@ -57,7 +57,7 @@ public abstract class PaginatedCollection<T> extends IterableWithMarker<T> {
 
    protected PaginatedCollection(List<T> items, Meta meta) {
       this.items = ImmutableList.copyOf(checkNotNull(items, "items cannot be null"));
-      this.meta = checkNotNull(meta, "meta cannot be null");
+      this.meta = meta;
    }
 
    public List<T> items() {
@@ -66,7 +66,7 @@ public abstract class PaginatedCollection<T> extends IterableWithMarker<T> {
 
    public Meta meta() {
       return meta;
-   }
+   }  
 
    @Override
    public Iterator<T> iterator() {
@@ -75,7 +75,7 @@ public abstract class PaginatedCollection<T> extends IterableWithMarker<T> {
 
    @Override
    public Optional<Object> nextMarker() {
-      if (meta.next() == null) {
+      if (meta == null || meta.next() == null) {
          return Optional.absent();
       }
       return Optional.fromNullable((Object) meta.next());


[5/9] jclouds-labs git commit: add AddApiVersionToRequest filter

Posted by na...@apache.org.
add AddApiVersionToRequest filter


Project: http://git-wip-us.apache.org/repos/asf/jclouds-labs/repo
Commit: http://git-wip-us.apache.org/repos/asf/jclouds-labs/commit/86bfeab8
Tree: http://git-wip-us.apache.org/repos/asf/jclouds-labs/tree/86bfeab8
Diff: http://git-wip-us.apache.org/repos/asf/jclouds-labs/diff/86bfeab8

Branch: refs/heads/2.0.x
Commit: 86bfeab8f9082759542d70b5d75d0d0235dddcec
Parents: b7f3923
Author: Andrea Turli <an...@gmail.com>
Authored: Thu Jan 12 10:21:24 2017 +0100
Committer: Ignasi Barrera <na...@apache.org>
Committed: Thu Feb 2 15:23:25 2017 +0100

----------------------------------------------------------------------
 .../org/jclouds/packet/PacketApiMetadata.java   |  1 +
 .../org/jclouds/packet/features/ProjectApi.java |  3 +-
 .../packet/filters/AddApiVersionToRequest.java  | 57 ++++++++++++++++++++
 .../compute/internal/BasePacketApiMockTest.java |  5 +-
 4 files changed, 63 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/86bfeab8/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java b/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
index 75ba0e6..e05685e 100644
--- a/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
+++ b/packet/src/main/java/org/jclouds/packet/PacketApiMetadata.java
@@ -65,6 +65,7 @@ public class PacketApiMetadata extends BaseHttpApiMetadata<PacketApi> {
                  .documentation(URI.create("https://www.packet.net/help/api/#"))
                  .defaultEndpoint("https://api.packet.net")
                  .defaultProperties(PacketApiMetadata.defaultProperties())
+                 .version("1")
                  //.view(typeToken(ComputeServiceContext.class))
                  .defaultModules(ImmutableSet.<Class<? extends Module>>builder()
                          .add(PacketHttpApiModule.class)

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/86bfeab8/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java b/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
index e6bf0ca..afdf1ef 100644
--- a/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
+++ b/packet/src/main/java/org/jclouds/packet/features/ProjectApi.java
@@ -36,6 +36,7 @@ import org.jclouds.packet.domain.Href;
 import org.jclouds.packet.domain.Project;
 import org.jclouds.packet.domain.internal.PaginatedCollection;
 import org.jclouds.packet.domain.options.ListOptions;
+import org.jclouds.packet.filters.AddApiVersionToRequest;
 import org.jclouds.packet.filters.AddXAuthTokenToRequest;
 import org.jclouds.packet.functions.BaseToPagedIterable;
 import org.jclouds.rest.annotations.Fallback;
@@ -49,7 +50,7 @@ import com.google.inject.TypeLiteral;
 
 @Path("/projects")
 @Consumes(MediaType.APPLICATION_JSON)
-@RequestFilters(AddXAuthTokenToRequest.class)
+@RequestFilters({ AddXAuthTokenToRequest.class, AddApiVersionToRequest.class} )
 public interface ProjectApi {
 
 

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/86bfeab8/packet/src/main/java/org/jclouds/packet/filters/AddApiVersionToRequest.java
----------------------------------------------------------------------
diff --git a/packet/src/main/java/org/jclouds/packet/filters/AddApiVersionToRequest.java b/packet/src/main/java/org/jclouds/packet/filters/AddApiVersionToRequest.java
new file mode 100644
index 0000000..70e66fb
--- /dev/null
+++ b/packet/src/main/java/org/jclouds/packet/filters/AddApiVersionToRequest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.filters;
+
+import java.util.Collection;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+
+import org.jclouds.http.HttpException;
+import org.jclouds.http.HttpRequest;
+import org.jclouds.http.HttpRequestFilter;
+import org.jclouds.rest.annotations.ApiVersion;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.net.HttpHeaders.ACCEPT;
+import static java.lang.String.format;
+
+@Singleton
+public class AddApiVersionToRequest implements HttpRequestFilter {
+
+    private final String apiVersion;
+
+    @Inject
+    AddApiVersionToRequest(@ApiVersion String apiVersion) {
+        this.apiVersion = apiVersion;
+    }
+
+    @Override
+    public HttpRequest filter(final HttpRequest request) throws HttpException {
+        Collection<String> accept = checkNotNull(request.getHeaders().get(ACCEPT), "accept header must not be null");
+        String versionHeader = Joiner.on("; ").join(ImmutableList.builder()
+                .addAll(accept)
+                .add(format("version=%s", apiVersion))
+                .build());
+        return request.toBuilder()
+                .replaceHeader(ACCEPT, versionHeader)
+                .build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/86bfeab8/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiMockTest.java b/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiMockTest.java
index 5b8d6ae..9fe3d6f 100644
--- a/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiMockTest.java
+++ b/packet/src/test/java/org/jclouds/packet/compute/internal/BasePacketApiMockTest.java
@@ -57,6 +57,7 @@ public class BasePacketApiMockTest {
    protected MockWebServer server;
    protected PacketApi api;
    private Json json;
+   private ApiContext<PacketApi> ctx;
    
    // So that we can ignore formatting.
    private final JsonParser parser = new JsonParser();
@@ -65,7 +66,7 @@ public class BasePacketApiMockTest {
    public void start() throws IOException {
       server = new MockWebServer();
       server.play();
-      ApiContext<PacketApi> ctx = ContextBuilder.newBuilder("packet")
+      ctx = ContextBuilder.newBuilder("packet")
             .credentials("", X_AUTHORIZATION_TOKEN)
             .endpoint(url(""))
             .modules(modules)
@@ -130,7 +131,7 @@ public class BasePacketApiMockTest {
       RecordedRequest request = server.takeRequest();
       assertEquals(request.getMethod(), method);
       assertEquals(request.getPath(), path);
-      assertEquals(request.getHeader("Accept"), "application/json");
+      assertEquals(request.getHeader("Accept"), "application/json; version=" + ctx.getMetadata().get("apiVersion"));
       assertEquals(request.getHeader("X-Auth-Token"), X_AUTHORIZATION_TOKEN);
       return request;
    }


[9/9] jclouds-labs git commit: Fix pom files for Packet

Posted by na...@apache.org.
Fix pom files for Packet


Project: http://git-wip-us.apache.org/repos/asf/jclouds-labs/repo
Commit: http://git-wip-us.apache.org/repos/asf/jclouds-labs/commit/5261cba2
Tree: http://git-wip-us.apache.org/repos/asf/jclouds-labs/tree/5261cba2
Diff: http://git-wip-us.apache.org/repos/asf/jclouds-labs/diff/5261cba2

Branch: refs/heads/2.0.x
Commit: 5261cba28f3a2696c538a62250e1f69c5875a971
Parents: 38f0a50
Author: Ignasi Barrera <na...@apache.org>
Authored: Thu Feb 2 15:34:04 2017 +0100
Committer: Ignasi Barrera <na...@apache.org>
Committed: Thu Feb 2 15:34:04 2017 +0100

----------------------------------------------------------------------
 packet/pom.xml | 2 +-
 pom.xml        | 1 +
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5261cba2/packet/pom.xml
----------------------------------------------------------------------
diff --git a/packet/pom.xml b/packet/pom.xml
index 8864894..2e4b72c 100644
--- a/packet/pom.xml
+++ b/packet/pom.xml
@@ -23,7 +23,7 @@
     <parent>
         <groupId>org.apache.jclouds.labs</groupId>
         <artifactId>jclouds-labs</artifactId>
-        <version>2.1.0-SNAPSHOT</version>
+        <version>2.0.1-SNAPSHOT</version>
     </parent>
 
     <!-- TODO: when out of labs, switch to org.jclouds.provider -->

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/5261cba2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c5241dc..b010cd2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -82,6 +82,7 @@
     <module>abiquo</module>
     <module>profitbricks-rest</module>
     <module>oneandone</module>
+    <module>packet</module>
     <module>vagrant</module>
   </modules>
 


[7/9] jclouds-labs git commit: Add remaining features to support the abstraction

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/SshKeyApiLiveTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/SshKeyApiLiveTest.java b/packet/src/test/java/org/jclouds/packet/features/SshKeyApiLiveTest.java
new file mode 100644
index 0000000..3ef4387
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/SshKeyApiLiveTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jclouds.packet.compute.internal.BasePacketApiLiveTest;
+import org.jclouds.packet.domain.SshKey;
+import org.jclouds.ssh.SshKeys;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertTrue;
+import static org.testng.util.Strings.isNullOrEmpty;
+
+@Test(groups = "live", testName = "SshKeyApiLiveTest")
+public class SshKeyApiLiveTest extends BasePacketApiLiveTest {
+
+   private SshKey sshKey;
+
+   public void testCreate() {
+      Map<String, String> keyPair = SshKeys.generate();
+      sshKey = api.sshKeyApi().create(prefix + "-sshkey-livetest", keyPair.get("public"));
+   }
+
+   @Test(dependsOnMethods = "testCreate")
+   public void testGet() {
+      api.sshKeyApi().get(sshKey.id());
+   }
+
+   @Test(dependsOnMethods = "testCreate")
+   public void testList() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(Iterables.all(api().list().concat(), new Predicate<SshKey>() {
+         @Override
+         public boolean apply(SshKey input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All ssh keys must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some ssh keys to be returned");
+   }
+
+   @Test(dependsOnMethods = "testCreate")
+   public void testListOnePage() {
+      final AtomicInteger found = new AtomicInteger(0);
+      assertTrue(api().list(page(1).perPage(5)).allMatch(new Predicate<SshKey>() {
+         @Override
+         public boolean apply(SshKey input) {
+            found.incrementAndGet();
+            return !isNullOrEmpty(input.id());
+         }
+      }), "All ssh keys must have the 'id' field populated");
+      assertTrue(found.get() > 0, "Expected some ssh keys to be returned");
+   }
+
+   @Test(dependsOnMethods = "testList", alwaysRun = true)
+   public void testDelete() throws InterruptedException {
+      api.sshKeyApi().delete(sshKey.id());
+   }
+
+   private SshKeyApi api() {
+      return api.sshKeyApi();
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/features/SshKeyApiMockTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/features/SshKeyApiMockTest.java b/packet/src/test/java/org/jclouds/packet/features/SshKeyApiMockTest.java
new file mode 100644
index 0000000..3bb1969
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/features/SshKeyApiMockTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.features;
+
+import org.jclouds.packet.compute.internal.BasePacketApiMockTest;
+import org.jclouds.packet.domain.SshKey;
+import org.testng.annotations.Test;
+
+import static com.google.common.collect.Iterables.isEmpty;
+import static com.google.common.collect.Iterables.size;
+import static org.jclouds.packet.domain.options.ListOptions.Builder.page;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+@Test(groups = "unit", testName = "SshKeyApiMockTest", singleThreaded = true)
+public class SshKeyApiMockTest extends BasePacketApiMockTest {
+
+   public void testListSshKeys() throws InterruptedException {
+      server.enqueue(jsonResponse("/sshKeys-first.json"));
+      server.enqueue(jsonResponse("/sshKeys-last.json"));
+
+      Iterable<SshKey> sshkeys = api.sshKeyApi().list().concat();
+
+      assertEquals(size(sshkeys), 8); // Force the PagedIterable to advance
+      assertEquals(server.getRequestCount(), 2);
+
+      assertSent(server, "GET", "/ssh-keys");
+      assertSent(server, "GET", "/ssh-keys?page=2");
+   }
+
+   public void testListSshKeysReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<SshKey> sshkeys = api.sshKeyApi().list().concat();
+
+      assertTrue(isEmpty(sshkeys));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/ssh-keys");
+   }
+
+   public void testListSshKeysWithOptions() throws InterruptedException {
+      server.enqueue(jsonResponse("/sshKeys-first.json"));
+
+      Iterable<SshKey> actions = api.sshKeyApi().list(page(1).perPage(5));
+
+      assertEquals(size(actions), 5);
+      assertEquals(server.getRequestCount(), 1);
+
+      assertSent(server, "GET", "/ssh-keys?page=1&per_page=5");
+   }
+
+   public void testListSshKeysWithOptionsReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      Iterable<SshKey> actions = api.sshKeyApi().list(page(1).perPage(5));
+
+      assertTrue(isEmpty(actions));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/ssh-keys?page=1&per_page=5");
+   }
+
+   public void testGetSshKey() throws InterruptedException {
+      server.enqueue(jsonResponse("/ssh-key.json"));
+
+      SshKey sshKey = api.sshKeyApi().get("1");
+
+      assertEquals(sshKey, objectFromResource("/ssh-key.json", SshKey.class));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/ssh-keys/1");
+   }
+
+   public void testGetSshKeyReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      SshKey sshKey = api.sshKeyApi().get("1");
+
+      assertNull(sshKey);
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "GET", "/ssh-keys/1");
+   }
+
+   public void testCreateSshKey() throws InterruptedException {
+      server.enqueue(jsonResponse("/ssh-key-create-res.json"));
+
+      SshKey sshKey = api.sshKeyApi().create(
+              "jclouds-ssh-key-livetest",
+              "ssh-rsa AAAAB3NzaC1yc2EAAAADAQABAAABAQCdgcoNzH4hCc0j3b4MuG503L/J54uyFvwCAOu8vSsYuLpJ4AEyEOv+T0SfdF605fK6GYXA16Rxk3lrPt7mfKGNtXR0Ripbv7Zc6PvCRorwgj/cjh/45miozjrkXAiHD1GFZycfbi4YsoWAqZj7W4mwtctmhrYM0FPdya2XoRpVy89N+A5Xo4Xtd6EZn6JGEKQM5+kF2aL3ggy0od/DqjuEVYwZoyTe1RgUTXZSU/Woh7WMhsRHbqd3eYz4s6ac8n8IJPGKtUaQeqUtH7OK6NRYXVypUrkqNlwdNYZAwrjXg/x5T3D+bo11LENASRt9OJ2OkmRSTqRxBeDkhnVauWK/"
+      );
+
+      assertEquals(sshKey, objectFromResource("/ssh-key-create-res.json", SshKey.class));
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "POST", "/ssh-keys", stringFromResource("/ssh-key-create-req.json"));
+   }
+
+   public void testDeleteSshKey() throws InterruptedException {
+      server.enqueue(response204());
+
+      api.sshKeyApi().delete("1");
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "DELETE", "/ssh-keys/1");
+   }
+
+   public void testDeleteSshKeyReturns404() throws InterruptedException {
+      server.enqueue(response404());
+
+      api.sshKeyApi().delete("1");
+
+      assertEquals(server.getRequestCount(), 1);
+      assertSent(server, "DELETE", "/ssh-keys/1");
+   }
+
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/functions/HrefToListOptionsTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/functions/HrefToListOptionsTest.java b/packet/src/test/java/org/jclouds/packet/functions/HrefToListOptionsTest.java
new file mode 100644
index 0000000..1fecb29
--- /dev/null
+++ b/packet/src/test/java/org/jclouds/packet/functions/HrefToListOptionsTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.
+ */
+package org.jclouds.packet.functions;
+
+import org.jclouds.packet.domain.Href;
+import org.jclouds.packet.domain.options.ListOptions;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Multimap;
+
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static org.jclouds.packet.domain.options.ListOptions.PAGE_PARAM;
+import static org.jclouds.packet.domain.options.ListOptions.PER_PAGE_PARAM;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
+
+@Test(groups = "unit", testName = "HrefToListOptionsTest")
+public class HrefToListOptionsTest {
+
+   public void testNoOptions() {
+      HrefToListOptions function = new HrefToListOptions();
+
+      ListOptions options = function.apply(Href.create("https://api.packet.net/projects"));
+      assertNotNull(options);
+
+      Multimap<String, String> params = options.buildQueryParameters();
+      assertFalse(params.containsKey(PAGE_PARAM));
+      assertFalse(params.containsKey(PER_PAGE_PARAM));
+   }
+
+   public void testWithOptions() {
+      HrefToListOptions function = new HrefToListOptions();
+
+      ListOptions options = function.apply(Href.create("https://api.packet.net/projects?page=2&per_page=5"));
+      assertNotNull(options);
+
+      Multimap<String, String> params = options.buildQueryParameters();
+      assertEquals(getOnlyElement(params.get(PAGE_PARAM)), "2");
+      assertEquals(getOnlyElement(params.get(PER_PAGE_PARAM)), "5");
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/java/org/jclouds/packet/functions/LinkToListOptionsTest.java
----------------------------------------------------------------------
diff --git a/packet/src/test/java/org/jclouds/packet/functions/LinkToListOptionsTest.java b/packet/src/test/java/org/jclouds/packet/functions/LinkToListOptionsTest.java
deleted file mode 100644
index 15262e8..0000000
--- a/packet/src/test/java/org/jclouds/packet/functions/LinkToListOptionsTest.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.
- */
-package org.jclouds.packet.functions;
-
-import org.jclouds.packet.domain.Href;
-import org.jclouds.packet.domain.options.ListOptions;
-import org.testng.annotations.Test;
-
-import com.google.common.collect.Multimap;
-
-import static com.google.common.collect.Iterables.getOnlyElement;
-import static org.jclouds.packet.domain.options.ListOptions.PAGE_PARAM;
-import static org.jclouds.packet.domain.options.ListOptions.PER_PAGE_PARAM;
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertFalse;
-import static org.testng.Assert.assertNotNull;
-
-@Test(groups = "unit", testName = "LinkToListOptionsTest")
-public class LinkToListOptionsTest {
-
-   public void testNoOptions() {
-      LinkToListOptions function = new LinkToListOptions();
-
-      ListOptions options = function.apply(Href.create("https://api.packet.net/projects"));
-      assertNotNull(options);
-
-      Multimap<String, String> params = options.buildQueryParameters();
-      assertFalse(params.containsKey(PAGE_PARAM));
-      assertFalse(params.containsKey(PER_PAGE_PARAM));
-   }
-
-   public void testWithOptions() {
-      LinkToListOptions function = new LinkToListOptions();
-
-      ListOptions options = function.apply(Href.create("https://api.packet.net/projects?page=2&per_page=5"));
-      assertNotNull(options);
-
-      Multimap<String, String> params = options.buildQueryParameters();
-      assertEquals(getOnlyElement(params.get(PAGE_PARAM)), "2");
-      assertEquals(getOnlyElement(params.get(PER_PAGE_PARAM)), "5");
-   }
-
-}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/device-create-req.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/device-create-req.json b/packet/src/test/resources/device-create-req.json
new file mode 100644
index 0000000..78e5ea5
--- /dev/null
+++ b/packet/src/test/resources/device-create-req.json
@@ -0,0 +1,11 @@
+{
+  "hostname": "jclouds-device-livetest",
+  "plan": "baremetal_0",
+  "billing_cycle": "hourly",
+  "facility": "ewr1",
+  "features": {},
+  "operating_system": "ubuntu_16_04",
+  "locked": false,
+  "userdata": "",
+  "tags": []
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/device-create-res.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/device-create-res.json b/packet/src/test/resources/device-create-res.json
new file mode 100644
index 0000000..7a5986f
--- /dev/null
+++ b/packet/src/test/resources/device-create-res.json
@@ -0,0 +1,211 @@
+{
+  "id": "3238ce59-fe02-4b9b-92c1-915ef4f3fb9e",
+  "short_id": "3238ce59",
+  "hostname": "andrea-device-livetest",
+  "description": null,
+  "state": "queued",
+  "tags": [],
+  "billing_cycle": "hourly",
+  "user": "root",
+  "iqn": "iqn.2017-01.net.packet:device.3238ce59",
+  "locked": false,
+  "bonding_mode": 5,
+  "created_at": "2017-01-20T14:15:28Z",
+  "updated_at": "2017-01-20T14:15:29Z",
+  "provisioning_percentage": 10.0,
+  "operating_system": {
+    "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+    "slug": "ubuntu_16_04",
+    "name": "Ubuntu 16.04 LTS",
+    "distro": "ubuntu",
+    "version": "16.04",
+    "provisionable_on": [
+      "baremetal_0",
+      "baremetal_1",
+      "baremetal_2",
+      "baremetal_2a",
+      "baremetal_3",
+      "baremetal_hua"
+    ]
+  },
+  "facility": {
+    "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+    "name": "Parsippany, NJ",
+    "code": "ewr1",
+    "features": [
+      "baremetal",
+      "storage"
+    ],
+    "address": null
+  },
+  "project": {
+    "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+  },
+  "ssh_keys": [
+    {
+      "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+    },
+    {
+      "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+    },
+    {
+      "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+    },
+    {
+      "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+    },
+    {
+      "href": "/ssh-keys/f82b69d7-8c7e-4a38-9283-ecdbcaba56aa"
+    },
+    {
+      "href": "/ssh-keys/da5d6c21-2e8c-43ac-820f-ff41bd4e6ebc"
+    }
+  ],
+  "project_lite": {
+    "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+  },
+  "volumes": [],
+  "ip_addresses": [],
+  "provisioning_events": [
+    {
+      "id": "32a4da09-37dd-446b-99ef-9f64c1eb0097",
+      "type": "provisioning.101",
+      "body": "Provisioning started",
+      "created_at": "2017-01-20T14:15:29Z",
+      "relationships": [
+        {
+          "href": "#5062a5fe-19ea-4b41-864c-e5f4dcb02fa0"
+        }
+      ],
+      "interpolated": "Provisioning started",
+      "href": "/events/32a4da09-37dd-446b-99ef-9f64c1eb0097"
+    },
+    {
+      "id": null,
+      "type": "provisioning.102",
+      "body": "Network configured",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Network configured"
+    },
+    {
+      "id": null,
+      "type": "provisioning.103",
+      "body": "Configuration written, restarting device",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Configuration written, restarting device"
+    },
+    {
+      "id": null,
+      "type": "provisioning.104",
+      "body": "Connected to magic install system",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Connected to magic install system"
+    },
+    {
+      "id": null,
+      "type": "provisioning.105",
+      "body": "Server partitions created",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Server partitions created"
+    },
+    {
+      "id": null,
+      "type": "provisioning.106",
+      "body": "Operating system packages installed",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Operating system packages installed"
+    },
+    {
+      "id": null,
+      "type": "provisioning.107",
+      "body": "Server networking interfaces configured",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Server networking interfaces configured"
+    },
+    {
+      "id": null,
+      "type": "provisioning.108",
+      "body": "Cloud-init packages installed and configured",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Cloud-init packages installed and configured"
+    },
+    {
+      "id": null,
+      "type": "provisioning.109",
+      "body": "Installation finished, rebooting server",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Installation finished, rebooting server"
+    },
+    {
+      "id": null,
+      "type": "provisioning.109",
+      "body": "Installation finished, rebooting server",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Installation finished, rebooting server"
+    }
+  ],
+  "plan": {
+    "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+    "slug": "baremetal_0",
+    "name": "Type 0",
+    "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+    "line": "baremetal",
+    "specs": {
+      "cpus": [
+        {
+          "count": 1,
+          "type": "Intel Atom C2550 @ 2.4Ghz"
+        }
+      ],
+      "memory": {
+        "total": "8GB"
+      },
+      "drives": [
+        {
+          "count": 1,
+          "size": "80GB",
+          "type": "SSD"
+        }
+      ],
+      "nics": [
+        {
+          "count": 2,
+          "type": "1Gbps"
+        }
+      ],
+      "features": {
+        "raid": false,
+        "txt": true
+      }
+    },
+    "available_in": [
+      {
+        "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+      },
+      {
+        "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+      },
+      {
+        "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+      },
+      {
+        "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+      }
+    ],
+    "pricing": {
+      "hour": 0.05
+    }
+  },
+  "userdata": "",
+  "root_password": "$n4hwka3!i",
+  "href": "/devices/3238ce59-fe02-4b9b-92c1-915ef4f3fb9e"
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/device.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/device.json b/packet/src/test/resources/device.json
new file mode 100644
index 0000000..b4daa39
--- /dev/null
+++ b/packet/src/test/resources/device.json
@@ -0,0 +1,278 @@
+{
+  "id": "98e22032-579e-4c04-bb12-05cc6a3864c8",
+  "short_id": "98e22032",
+  "hostname": "test",
+  "description": null,
+  "state": "provisioning",
+  "tags": [],
+  "billing_cycle": "hourly",
+  "user": "root",
+  "iqn": "iqn.2017-01.net.packet:device.98e22032",
+  "locked": false,
+  "bonding_mode": 5,
+  "created_at": "2017-01-03T09:47:59Z",
+  "updated_at": "2017-01-03T09:50:13Z",
+  "provisioning_percentage": 50,
+  "operating_system": {
+    "id": "68bad60d-f5a7-45c2-ad09-573edaad3a3c",
+    "slug": "centos_7",
+    "name": "Centos 7",
+    "distro": "centos",
+    "version": "7",
+    "provisionable_on": [
+      "baremetal_0",
+      "baremetal_1",
+      "baremetal_2",
+      "baremetal_3"
+    ]
+  },
+  "facility": {
+    "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+    "name": "Parsippany, NJ",
+    "code": "ewr1",
+    "features": [
+      "baremetal",
+      "storage"
+    ],
+    "address": null
+  },
+  "project": {
+    "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+  },
+  "ssh_keys": [
+    {
+      "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+    },
+    {
+      "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+    },
+    {
+      "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+    },
+    {
+      "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+    },
+    {
+      "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+    }
+  ],
+  "project_lite": {
+    "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+  },
+  "volumes": [],
+  "ip_addresses": [
+    {
+      "id": "5d0262c7-1727-411c-94c4-9e6f15490dd3",
+      "address_family": 4,
+      "netmask": "255.255.255.254",
+      "created_at": "2017-01-03T09:47:59Z",
+      "public": true,
+      "cidr": 31,
+      "management": true,
+      "manageable": true,
+      "assigned_to": {
+        "href": "/devices/98e22032-579e-4c04-bb12-05cc6a3864c8"
+      },
+      "network": "147.75.106.80",
+      "address": "147.75.106.81",
+      "gateway": "147.75.106.80",
+      "href": "/ips/5d0262c7-1727-411c-94c4-9e6f15490dd3"
+    },
+    {
+      "id": "f7d0e65c-eb3b-42bd-af9b-ad3a736d8d43",
+      "address_family": 6,
+      "netmask": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe",
+      "created_at": "2017-01-03T09:47:59Z",
+      "public": true,
+      "cidr": 127,
+      "management": true,
+      "manageable": true,
+      "assigned_to": {
+        "href": "/devices/98e22032-579e-4c04-bb12-05cc6a3864c8"
+      },
+      "network": "2604:1380:2:9800::",
+      "address": "2604:1380:2:9800::1",
+      "gateway": "2604:1380:2:9800::",
+      "href": "/ips/f7d0e65c-eb3b-42bd-af9b-ad3a736d8d43"
+    },
+    {
+      "id": "a3d00b4e-d74f-4ac2-8bc9-91065d815b41",
+      "address_family": 4,
+      "netmask": "255.255.255.254",
+      "created_at": "2017-01-03T09:47:59Z",
+      "public": false,
+      "cidr": 31,
+      "management": true,
+      "manageable": true,
+      "assigned_to": {
+        "href": "/devices/98e22032-579e-4c04-bb12-05cc6a3864c8"
+      },
+      "network": "10.99.214.0",
+      "address": "10.99.214.1",
+      "gateway": "10.99.214.0",
+      "href": "/ips/a3d00b4e-d74f-4ac2-8bc9-91065d815b41"
+    }
+  ],
+  "provisioning_events": [
+    {
+      "id": "bd62123b-afed-4e54-b1b9-89e219ba9cf0",
+      "type": "provisioning.101",
+      "body": "Provisioning started",
+      "created_at": "2017-01-03T09:47:59Z",
+      "relationships": [
+        {
+          "href": "#81909921-255e-413c-883a-c58d14c801ae"
+        }
+      ],
+      "interpolated": "Provisioning started",
+      "href": "/events/bd62123b-afed-4e54-b1b9-89e219ba9cf0"
+    },
+    {
+      "id": "14b4a9e6-be90-40ee-be48-b272f855e39c",
+      "type": "provisioning.102",
+      "body": "Network configured with addresses 147.75.106.81, 2604:1380:2:9800::1, and 10.99.214.1",
+      "created_at": "2017-01-03T09:48:46Z",
+      "relationships": [
+        {
+          "href": "#28588657-b8bf-44a1-98d3-27f8e04b660a"
+        }
+      ],
+      "interpolated": "Network configured with addresses 147.75.106.81, 2604:1380:2:9800::1, and 10.99.214.1",
+      "href": "/events/14b4a9e6-be90-40ee-be48-b272f855e39c"
+    },
+    {
+      "id": "57e3cadb-f9aa-4c73-be54-3c83e6cf462e",
+      "type": "provisioning.103",
+      "body": "Configuration written, restarting device",
+      "created_at": "2017-01-03T09:48:57Z",
+      "relationships": [
+        {
+          "href": "#25a7e807-f4aa-4f36-83cd-8347baeb26bc"
+        }
+      ],
+      "interpolated": "Configuration written, restarting device",
+      "href": "/events/57e3cadb-f9aa-4c73-be54-3c83e6cf462e"
+    },
+    {
+      "id": "b8322996-f57e-4c87-96cc-a16f33a0c305",
+      "type": "provisioning.104",
+      "body": "Connected to magic install system",
+      "created_at": "2017-01-03T09:50:13Z",
+      "relationships": [
+        {
+          "href": "#a4825d20-f7f2-426a-88db-38696bd3dfd6"
+        }
+      ],
+      "interpolated": "Connected to magic install system",
+      "href": "/events/b8322996-f57e-4c87-96cc-a16f33a0c305"
+    },
+    {
+      "id": "f00a00a9-7c59-420d-bdcf-2c0993303cf6",
+      "type": "provisioning.105",
+      "body": "Server partitions created",
+      "created_at": "2017-01-03T09:50:13Z",
+      "relationships": [
+        {
+          "href": "#6140659b-5e3d-4686-be8f-8879b6d3e27f"
+        }
+      ],
+      "interpolated": "Server partitions created",
+      "href": "/events/f00a00a9-7c59-420d-bdcf-2c0993303cf6"
+    },
+    {
+      "id": null,
+      "type": "provisioning.106",
+      "body": "Operating system packages installed",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Operating system packages installed"
+    },
+    {
+      "id": null,
+      "type": "provisioning.107",
+      "body": "Server networking interfaces configured",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Server networking interfaces configured"
+    },
+    {
+      "id": null,
+      "type": "provisioning.108",
+      "body": "Cloud-init packages installed and configured",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Cloud-init packages installed and configured"
+    },
+    {
+      "id": null,
+      "type": "provisioning.109",
+      "body": "Installation finished, rebooting server",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Installation finished, rebooting server"
+    },
+    {
+      "id": null,
+      "type": "provisioning.109",
+      "body": "Installation finished, rebooting server",
+      "created_at": null,
+      "relationships": [],
+      "interpolated": "Installation finished, rebooting server"
+    }
+  ],
+  "plan": {
+    "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+    "slug": "baremetal_0",
+    "name": "Type 0",
+    "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+    "line": "baremetal",
+    "specs": {
+      "cpus": [
+        {
+          "count": 1,
+          "type": "Intel Atom C2550 @ 2.4Ghz"
+        }
+      ],
+      "memory": {
+        "total": "8GB"
+      },
+      "drives": [
+        {
+          "count": 1,
+          "size": "80GB",
+          "type": "SSD"
+        }
+      ],
+      "nics": [
+        {
+          "count": 2,
+          "type": "1Gbps"
+        }
+      ],
+      "features": {
+        "raid": false,
+        "txt": true
+      }
+    },
+    "available_in": [
+      {
+        "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+      },
+      {
+        "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+      },
+      {
+        "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+      },
+      {
+        "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+      }
+    ],
+    "pricing": {
+      "hour": 0.05
+    }
+  },
+  "userdata": "",
+  "root_password": ",q4*a8(eny",
+  "href": "/devices/98e22032-579e-4c04-bb12-05cc6a3864c8"
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/devices-first.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/devices-first.json b/packet/src/test/resources/devices-first.json
new file mode 100644
index 0000000..514c0eb
--- /dev/null
+++ b/packet/src/test/resources/devices-first.json
@@ -0,0 +1,910 @@
+{
+  "devices": [
+    {
+      "id": "5bd27259-8b71-48f7-879d-c4e695cb2f31",
+      "short_id": "5bd27259",
+      "hostname": "andrea-device-livetest",
+      "description": null,
+      "state": "active",
+      "tags": [],
+      "billing_cycle": "hourly",
+      "user": "root",
+      "iqn": "iqn.2017-01.net.packet:device.5bd27259",
+      "locked": false,
+      "bonding_mode": 5,
+      "created_at": "2017-01-18T17:12:29Z",
+      "updated_at": "2017-01-18T17:16:29Z",
+      "operating_system": {
+        "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+        "slug": "ubuntu_16_04",
+        "name": "Ubuntu 16.04 LTS",
+        "distro": "ubuntu",
+        "version": "16.04",
+        "provisionable_on": [
+          "baremetal_0",
+          "baremetal_1",
+          "baremetal_2",
+          "baremetal_2a",
+          "baremetal_3",
+          "baremetal_hua"
+        ]
+      },
+      "facility": {
+        "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+        "name": "Parsippany, NJ",
+        "code": "ewr1",
+        "features": [
+          "baremetal",
+          "storage"
+        ],
+        "address": null
+      },
+      "project": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+        },
+        {
+          "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+        },
+        {
+          "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+        },
+        {
+          "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+        },
+        {
+          "href": "/ssh-keys/f82b69d7-8c7e-4a38-9283-ecdbcaba56aa"
+        },
+        {
+          "href": "/ssh-keys/cf9db551-ec79-4c16-a776-97cdbcb3cec4"
+        }
+      ],
+      "project_lite": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "volumes": [],
+      "ip_addresses": [
+        {
+          "id": "cbff502b-12ca-4986-b683-2cb4e119ae40",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "147.75.197.54",
+          "address": "147.75.197.55",
+          "gateway": "147.75.197.54",
+          "href": "/ips/cbff502b-12ca-4986-b683-2cb4e119ae40"
+        },
+        {
+          "id": "d5cec621-18c1-4e02-8f71-8dd46370c3e5",
+          "address_family": 6,
+          "netmask": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 127,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "2604:1380:2:9800::",
+          "address": "2604:1380:2:9800::1",
+          "gateway": "2604:1380:2:9800::",
+          "href": "/ips/d5cec621-18c1-4e02-8f71-8dd46370c3e5"
+        },
+        {
+          "id": "6d206311-1f88-454d-9cf2-afca79f51fa3",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": false,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "10.99.214.0",
+          "address": "10.99.214.1",
+          "gateway": "10.99.214.0",
+          "href": "/ips/6d206311-1f88-454d-9cf2-afca79f51fa3"
+        }
+      ],
+      "plan": {
+        "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+        "slug": "baremetal_0",
+        "name": "Type 0",
+        "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+        "line": "baremetal",
+        "specs": {
+          "cpus": [
+            {
+              "count": 1,
+              "type": "Intel Atom C2550 @ 2.4Ghz"
+            }
+          ],
+          "memory": {
+            "total": "8GB"
+          },
+          "drives": [
+            {
+              "count": 1,
+              "size": "80GB",
+              "type": "SSD"
+            }
+          ],
+          "nics": [
+            {
+              "count": 2,
+              "type": "1Gbps"
+            }
+          ],
+          "features": {
+            "raid": false,
+            "txt": true
+          }
+        },
+        "available_in": [
+          {
+            "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+          },
+          {
+            "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+          },
+          {
+            "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+          },
+          {
+            "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+          }
+        ],
+        "pricing": {
+          "hour": 0.05
+        }
+      },
+      "userdata": "",
+      "root_password": "soaz0#a=cd",
+      "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+    },
+    {
+      "id": "5bd27259-8b71-48f7-879d-c4e695cb2f32",
+      "short_id": "5bd27259",
+      "hostname": "andrea-device-livetest",
+      "description": null,
+      "state": "active",
+      "tags": [],
+      "billing_cycle": "hourly",
+      "user": "root",
+      "iqn": "iqn.2017-01.net.packet:device.5bd27259",
+      "locked": false,
+      "bonding_mode": 5,
+      "created_at": "2017-01-18T17:12:29Z",
+      "updated_at": "2017-01-18T17:16:29Z",
+      "operating_system": {
+        "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+        "slug": "ubuntu_16_04",
+        "name": "Ubuntu 16.04 LTS",
+        "distro": "ubuntu",
+        "version": "16.04",
+        "provisionable_on": [
+          "baremetal_0",
+          "baremetal_1",
+          "baremetal_2",
+          "baremetal_2a",
+          "baremetal_3",
+          "baremetal_hua"
+        ]
+      },
+      "facility": {
+        "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+        "name": "Parsippany, NJ",
+        "code": "ewr1",
+        "features": [
+          "baremetal",
+          "storage"
+        ],
+        "address": null
+      },
+      "project": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+        },
+        {
+          "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+        },
+        {
+          "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+        },
+        {
+          "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+        },
+        {
+          "href": "/ssh-keys/f82b69d7-8c7e-4a38-9283-ecdbcaba56aa"
+        },
+        {
+          "href": "/ssh-keys/cf9db551-ec79-4c16-a776-97cdbcb3cec4"
+        }
+      ],
+      "project_lite": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "volumes": [],
+      "ip_addresses": [
+        {
+          "id": "cbff502b-12ca-4986-b683-2cb4e119ae40",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "147.75.197.54",
+          "address": "147.75.197.55",
+          "gateway": "147.75.197.54",
+          "href": "/ips/cbff502b-12ca-4986-b683-2cb4e119ae40"
+        },
+        {
+          "id": "d5cec621-18c1-4e02-8f71-8dd46370c3e5",
+          "address_family": 6,
+          "netmask": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 127,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "2604:1380:2:9800::",
+          "address": "2604:1380:2:9800::1",
+          "gateway": "2604:1380:2:9800::",
+          "href": "/ips/d5cec621-18c1-4e02-8f71-8dd46370c3e5"
+        },
+        {
+          "id": "6d206311-1f88-454d-9cf2-afca79f51fa3",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": false,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "10.99.214.0",
+          "address": "10.99.214.1",
+          "gateway": "10.99.214.0",
+          "href": "/ips/6d206311-1f88-454d-9cf2-afca79f51fa3"
+        }
+      ],
+      "plan": {
+        "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+        "slug": "baremetal_0",
+        "name": "Type 0",
+        "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+        "line": "baremetal",
+        "specs": {
+          "cpus": [
+            {
+              "count": 1,
+              "type": "Intel Atom C2550 @ 2.4Ghz"
+            }
+          ],
+          "memory": {
+            "total": "8GB"
+          },
+          "drives": [
+            {
+              "count": 1,
+              "size": "80GB",
+              "type": "SSD"
+            }
+          ],
+          "nics": [
+            {
+              "count": 2,
+              "type": "1Gbps"
+            }
+          ],
+          "features": {
+            "raid": false,
+            "txt": true
+          }
+        },
+        "available_in": [
+          {
+            "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+          },
+          {
+            "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+          },
+          {
+            "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+          },
+          {
+            "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+          }
+        ],
+        "pricing": {
+          "hour": 0.05
+        }
+      },
+      "userdata": "",
+      "root_password": "soaz0#a=cd",
+      "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+    },
+    {
+      "id": "5bd27259-8b71-48f7-879d-c4e695cb2f33",
+      "short_id": "5bd27259",
+      "hostname": "andrea-device-livetest",
+      "description": null,
+      "state": "active",
+      "tags": [],
+      "billing_cycle": "hourly",
+      "user": "root",
+      "iqn": "iqn.2017-01.net.packet:device.5bd27259",
+      "locked": false,
+      "bonding_mode": 5,
+      "created_at": "2017-01-18T17:12:29Z",
+      "updated_at": "2017-01-18T17:16:29Z",
+      "operating_system": {
+        "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+        "slug": "ubuntu_16_04",
+        "name": "Ubuntu 16.04 LTS",
+        "distro": "ubuntu",
+        "version": "16.04",
+        "provisionable_on": [
+          "baremetal_0",
+          "baremetal_1",
+          "baremetal_2",
+          "baremetal_2a",
+          "baremetal_3",
+          "baremetal_hua"
+        ]
+      },
+      "facility": {
+        "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+        "name": "Parsippany, NJ",
+        "code": "ewr1",
+        "features": [
+          "baremetal",
+          "storage"
+        ],
+        "address": null
+      },
+      "project": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+        },
+        {
+          "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+        },
+        {
+          "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+        },
+        {
+          "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+        },
+        {
+          "href": "/ssh-keys/f82b69d7-8c7e-4a38-9283-ecdbcaba56aa"
+        },
+        {
+          "href": "/ssh-keys/cf9db551-ec79-4c16-a776-97cdbcb3cec4"
+        }
+      ],
+      "project_lite": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "volumes": [],
+      "ip_addresses": [
+        {
+          "id": "cbff502b-12ca-4986-b683-2cb4e119ae40",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "147.75.197.54",
+          "address": "147.75.197.55",
+          "gateway": "147.75.197.54",
+          "href": "/ips/cbff502b-12ca-4986-b683-2cb4e119ae40"
+        },
+        {
+          "id": "d5cec621-18c1-4e02-8f71-8dd46370c3e5",
+          "address_family": 6,
+          "netmask": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 127,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "2604:1380:2:9800::",
+          "address": "2604:1380:2:9800::1",
+          "gateway": "2604:1380:2:9800::",
+          "href": "/ips/d5cec621-18c1-4e02-8f71-8dd46370c3e5"
+        },
+        {
+          "id": "6d206311-1f88-454d-9cf2-afca79f51fa3",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": false,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "10.99.214.0",
+          "address": "10.99.214.1",
+          "gateway": "10.99.214.0",
+          "href": "/ips/6d206311-1f88-454d-9cf2-afca79f51fa3"
+        }
+      ],
+      "plan": {
+        "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+        "slug": "baremetal_0",
+        "name": "Type 0",
+        "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+        "line": "baremetal",
+        "specs": {
+          "cpus": [
+            {
+              "count": 1,
+              "type": "Intel Atom C2550 @ 2.4Ghz"
+            }
+          ],
+          "memory": {
+            "total": "8GB"
+          },
+          "drives": [
+            {
+              "count": 1,
+              "size": "80GB",
+              "type": "SSD"
+            }
+          ],
+          "nics": [
+            {
+              "count": 2,
+              "type": "1Gbps"
+            }
+          ],
+          "features": {
+            "raid": false,
+            "txt": true
+          }
+        },
+        "available_in": [
+          {
+            "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+          },
+          {
+            "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+          },
+          {
+            "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+          },
+          {
+            "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+          }
+        ],
+        "pricing": {
+          "hour": 0.05
+        }
+      },
+      "userdata": "",
+      "root_password": "soaz0#a=cd",
+      "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+    },
+    {
+      "id": "5bd27259-8b71-48f7-879d-c4e695cb2f34",
+      "short_id": "5bd27259",
+      "hostname": "andrea-device-livetest",
+      "description": null,
+      "state": "active",
+      "tags": [],
+      "billing_cycle": "hourly",
+      "user": "root",
+      "iqn": "iqn.2017-01.net.packet:device.5bd27259",
+      "locked": false,
+      "bonding_mode": 5,
+      "created_at": "2017-01-18T17:12:29Z",
+      "updated_at": "2017-01-18T17:16:29Z",
+      "operating_system": {
+        "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+        "slug": "ubuntu_16_04",
+        "name": "Ubuntu 16.04 LTS",
+        "distro": "ubuntu",
+        "version": "16.04",
+        "provisionable_on": [
+          "baremetal_0",
+          "baremetal_1",
+          "baremetal_2",
+          "baremetal_2a",
+          "baremetal_3",
+          "baremetal_hua"
+        ]
+      },
+      "facility": {
+        "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+        "name": "Parsippany, NJ",
+        "code": "ewr1",
+        "features": [
+          "baremetal",
+          "storage"
+        ],
+        "address": null
+      },
+      "project": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+        },
+        {
+          "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+        },
+        {
+          "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+        },
+        {
+          "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+        },
+        {
+          "href": "/ssh-keys/f82b69d7-8c7e-4a38-9283-ecdbcaba56aa"
+        },
+        {
+          "href": "/ssh-keys/cf9db551-ec79-4c16-a776-97cdbcb3cec4"
+        }
+      ],
+      "project_lite": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "volumes": [],
+      "ip_addresses": [
+        {
+          "id": "cbff502b-12ca-4986-b683-2cb4e119ae40",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "147.75.197.54",
+          "address": "147.75.197.55",
+          "gateway": "147.75.197.54",
+          "href": "/ips/cbff502b-12ca-4986-b683-2cb4e119ae40"
+        },
+        {
+          "id": "d5cec621-18c1-4e02-8f71-8dd46370c3e5",
+          "address_family": 6,
+          "netmask": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 127,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "2604:1380:2:9800::",
+          "address": "2604:1380:2:9800::1",
+          "gateway": "2604:1380:2:9800::",
+          "href": "/ips/d5cec621-18c1-4e02-8f71-8dd46370c3e5"
+        },
+        {
+          "id": "6d206311-1f88-454d-9cf2-afca79f51fa3",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": false,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "10.99.214.0",
+          "address": "10.99.214.1",
+          "gateway": "10.99.214.0",
+          "href": "/ips/6d206311-1f88-454d-9cf2-afca79f51fa3"
+        }
+      ],
+      "plan": {
+        "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+        "slug": "baremetal_0",
+        "name": "Type 0",
+        "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+        "line": "baremetal",
+        "specs": {
+          "cpus": [
+            {
+              "count": 1,
+              "type": "Intel Atom C2550 @ 2.4Ghz"
+            }
+          ],
+          "memory": {
+            "total": "8GB"
+          },
+          "drives": [
+            {
+              "count": 1,
+              "size": "80GB",
+              "type": "SSD"
+            }
+          ],
+          "nics": [
+            {
+              "count": 2,
+              "type": "1Gbps"
+            }
+          ],
+          "features": {
+            "raid": false,
+            "txt": true
+          }
+        },
+        "available_in": [
+          {
+            "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+          },
+          {
+            "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+          },
+          {
+            "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+          },
+          {
+            "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+          }
+        ],
+        "pricing": {
+          "hour": 0.05
+        }
+      },
+      "userdata": "",
+      "root_password": "soaz0#a=cd",
+      "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+    },
+    {
+      "id": "5bd27259-8b71-48f7-879d-c4e695cb2f35",
+      "short_id": "5bd27259",
+      "hostname": "andrea-device-livetest",
+      "description": null,
+      "state": "active",
+      "tags": [],
+      "billing_cycle": "hourly",
+      "user": "root",
+      "iqn": "iqn.2017-01.net.packet:device.5bd27259",
+      "locked": false,
+      "bonding_mode": 5,
+      "created_at": "2017-01-18T17:12:29Z",
+      "updated_at": "2017-01-18T17:16:29Z",
+      "operating_system": {
+        "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+        "slug": "ubuntu_16_04",
+        "name": "Ubuntu 16.04 LTS",
+        "distro": "ubuntu",
+        "version": "16.04",
+        "provisionable_on": [
+          "baremetal_0",
+          "baremetal_1",
+          "baremetal_2",
+          "baremetal_2a",
+          "baremetal_3",
+          "baremetal_hua"
+        ]
+      },
+      "facility": {
+        "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+        "name": "Parsippany, NJ",
+        "code": "ewr1",
+        "features": [
+          "baremetal",
+          "storage"
+        ],
+        "address": null
+      },
+      "project": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+        },
+        {
+          "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+        },
+        {
+          "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+        },
+        {
+          "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+        },
+        {
+          "href": "/ssh-keys/f82b69d7-8c7e-4a38-9283-ecdbcaba56aa"
+        },
+        {
+          "href": "/ssh-keys/cf9db551-ec79-4c16-a776-97cdbcb3cec4"
+        }
+      ],
+      "project_lite": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "volumes": [],
+      "ip_addresses": [
+        {
+          "id": "cbff502b-12ca-4986-b683-2cb4e119ae40",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "147.75.197.54",
+          "address": "147.75.197.55",
+          "gateway": "147.75.197.54",
+          "href": "/ips/cbff502b-12ca-4986-b683-2cb4e119ae40"
+        },
+        {
+          "id": "d5cec621-18c1-4e02-8f71-8dd46370c3e5",
+          "address_family": 6,
+          "netmask": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 127,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "2604:1380:2:9800::",
+          "address": "2604:1380:2:9800::1",
+          "gateway": "2604:1380:2:9800::",
+          "href": "/ips/d5cec621-18c1-4e02-8f71-8dd46370c3e5"
+        },
+        {
+          "id": "6d206311-1f88-454d-9cf2-afca79f51fa3",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": false,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "10.99.214.0",
+          "address": "10.99.214.1",
+          "gateway": "10.99.214.0",
+          "href": "/ips/6d206311-1f88-454d-9cf2-afca79f51fa3"
+        }
+      ],
+      "plan": {
+        "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+        "slug": "baremetal_0",
+        "name": "Type 0",
+        "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+        "line": "baremetal",
+        "specs": {
+          "cpus": [
+            {
+              "count": 1,
+              "type": "Intel Atom C2550 @ 2.4Ghz"
+            }
+          ],
+          "memory": {
+            "total": "8GB"
+          },
+          "drives": [
+            {
+              "count": 1,
+              "size": "80GB",
+              "type": "SSD"
+            }
+          ],
+          "nics": [
+            {
+              "count": 2,
+              "type": "1Gbps"
+            }
+          ],
+          "features": {
+            "raid": false,
+            "txt": true
+          }
+        },
+        "available_in": [
+          {
+            "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+          },
+          {
+            "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+          },
+          {
+            "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+          },
+          {
+            "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+          }
+        ],
+        "pricing": {
+          "hour": 0.05
+        }
+      },
+      "userdata": "",
+      "root_password": "soaz0#a=cd",
+      "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=1"
+    },
+    "previous": null,
+    "self": {
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=1"
+    },
+    "next": {
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=2"
+    },
+    "last": {
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=2"
+    },
+    "total": 7
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/devices-last.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/devices-last.json b/packet/src/test/resources/devices-last.json
new file mode 100644
index 0000000..55ef2f0
--- /dev/null
+++ b/packet/src/test/resources/devices-last.json
@@ -0,0 +1,376 @@
+{
+  "devices": [
+    {
+      "id": "5bd27259-8b71-48f7-879d-c4e695cb2f36",
+      "short_id": "5bd27259",
+      "hostname": "andrea-device-livetest",
+      "description": null,
+      "state": "active",
+      "tags": [],
+      "billing_cycle": "hourly",
+      "user": "root",
+      "iqn": "iqn.2017-01.net.packet:device.5bd27259",
+      "locked": false,
+      "bonding_mode": 5,
+      "created_at": "2017-01-18T17:12:29Z",
+      "updated_at": "2017-01-18T17:16:29Z",
+      "operating_system": {
+        "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+        "slug": "ubuntu_16_04",
+        "name": "Ubuntu 16.04 LTS",
+        "distro": "ubuntu",
+        "version": "16.04",
+        "provisionable_on": [
+          "baremetal_0",
+          "baremetal_1",
+          "baremetal_2",
+          "baremetal_2a",
+          "baremetal_3",
+          "baremetal_hua"
+        ]
+      },
+      "facility": {
+        "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+        "name": "Parsippany, NJ",
+        "code": "ewr1",
+        "features": [
+          "baremetal",
+          "storage"
+        ],
+        "address": null
+      },
+      "project": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+        },
+        {
+          "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+        },
+        {
+          "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+        },
+        {
+          "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+        },
+        {
+          "href": "/ssh-keys/f82b69d7-8c7e-4a38-9283-ecdbcaba56aa"
+        },
+        {
+          "href": "/ssh-keys/cf9db551-ec79-4c16-a776-97cdbcb3cec4"
+        }
+      ],
+      "project_lite": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "volumes": [],
+      "ip_addresses": [
+        {
+          "id": "cbff502b-12ca-4986-b683-2cb4e119ae40",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "147.75.197.54",
+          "address": "147.75.197.55",
+          "gateway": "147.75.197.54",
+          "href": "/ips/cbff502b-12ca-4986-b683-2cb4e119ae40"
+        },
+        {
+          "id": "d5cec621-18c1-4e02-8f71-8dd46370c3e5",
+          "address_family": 6,
+          "netmask": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 127,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "2604:1380:2:9800::",
+          "address": "2604:1380:2:9800::1",
+          "gateway": "2604:1380:2:9800::",
+          "href": "/ips/d5cec621-18c1-4e02-8f71-8dd46370c3e5"
+        },
+        {
+          "id": "6d206311-1f88-454d-9cf2-afca79f51fa3",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": false,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "10.99.214.0",
+          "address": "10.99.214.1",
+          "gateway": "10.99.214.0",
+          "href": "/ips/6d206311-1f88-454d-9cf2-afca79f51fa3"
+        }
+      ],
+      "plan": {
+        "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+        "slug": "baremetal_0",
+        "name": "Type 0",
+        "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+        "line": "baremetal",
+        "specs": {
+          "cpus": [
+            {
+              "count": 1,
+              "type": "Intel Atom C2550 @ 2.4Ghz"
+            }
+          ],
+          "memory": {
+            "total": "8GB"
+          },
+          "drives": [
+            {
+              "count": 1,
+              "size": "80GB",
+              "type": "SSD"
+            }
+          ],
+          "nics": [
+            {
+              "count": 2,
+              "type": "1Gbps"
+            }
+          ],
+          "features": {
+            "raid": false,
+            "txt": true
+          }
+        },
+        "available_in": [
+          {
+            "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+          },
+          {
+            "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+          },
+          {
+            "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+          },
+          {
+            "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+          }
+        ],
+        "pricing": {
+          "hour": 0.05
+        }
+      },
+      "userdata": "",
+      "root_password": "soaz0#a=cd",
+      "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+    },
+    {
+      "id": "5bd27259-8b71-48f7-879d-c4e695cb2f37",
+      "short_id": "5bd27259",
+      "hostname": "andrea-device-livetest",
+      "description": null,
+      "state": "active",
+      "tags": [],
+      "billing_cycle": "hourly",
+      "user": "root",
+      "iqn": "iqn.2017-01.net.packet:device.5bd27259",
+      "locked": false,
+      "bonding_mode": 5,
+      "created_at": "2017-01-18T17:12:29Z",
+      "updated_at": "2017-01-18T17:16:29Z",
+      "operating_system": {
+        "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+        "slug": "ubuntu_16_04",
+        "name": "Ubuntu 16.04 LTS",
+        "distro": "ubuntu",
+        "version": "16.04",
+        "provisionable_on": [
+          "baremetal_0",
+          "baremetal_1",
+          "baremetal_2",
+          "baremetal_2a",
+          "baremetal_3",
+          "baremetal_hua"
+        ]
+      },
+      "facility": {
+        "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+        "name": "Parsippany, NJ",
+        "code": "ewr1",
+        "features": [
+          "baremetal",
+          "storage"
+        ],
+        "address": null
+      },
+      "project": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+        },
+        {
+          "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+        },
+        {
+          "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+        },
+        {
+          "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+        },
+        {
+          "href": "/ssh-keys/f82b69d7-8c7e-4a38-9283-ecdbcaba56aa"
+        },
+        {
+          "href": "/ssh-keys/cf9db551-ec79-4c16-a776-97cdbcb3cec4"
+        }
+      ],
+      "project_lite": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "volumes": [],
+      "ip_addresses": [
+        {
+          "id": "cbff502b-12ca-4986-b683-2cb4e119ae40",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "147.75.197.54",
+          "address": "147.75.197.55",
+          "gateway": "147.75.197.54",
+          "href": "/ips/cbff502b-12ca-4986-b683-2cb4e119ae40"
+        },
+        {
+          "id": "d5cec621-18c1-4e02-8f71-8dd46370c3e5",
+          "address_family": 6,
+          "netmask": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": true,
+          "cidr": 127,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "2604:1380:2:9800::",
+          "address": "2604:1380:2:9800::1",
+          "gateway": "2604:1380:2:9800::",
+          "href": "/ips/d5cec621-18c1-4e02-8f71-8dd46370c3e5"
+        },
+        {
+          "id": "6d206311-1f88-454d-9cf2-afca79f51fa3",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-18T17:12:30Z",
+          "public": false,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "enabled": true,
+          "assigned_to": {
+            "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+          },
+          "network": "10.99.214.0",
+          "address": "10.99.214.1",
+          "gateway": "10.99.214.0",
+          "href": "/ips/6d206311-1f88-454d-9cf2-afca79f51fa3"
+        }
+      ],
+      "plan": {
+        "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+        "slug": "baremetal_0",
+        "name": "Type 0",
+        "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+        "line": "baremetal",
+        "specs": {
+          "cpus": [
+            {
+              "count": 1,
+              "type": "Intel Atom C2550 @ 2.4Ghz"
+            }
+          ],
+          "memory": {
+            "total": "8GB"
+          },
+          "drives": [
+            {
+              "count": 1,
+              "size": "80GB",
+              "type": "SSD"
+            }
+          ],
+          "nics": [
+            {
+              "count": 2,
+              "type": "1Gbps"
+            }
+          ],
+          "features": {
+            "raid": false,
+            "txt": true
+          }
+        },
+        "available_in": [
+          {
+            "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+          },
+          {
+            "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+          },
+          {
+            "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+          },
+          {
+            "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+          }
+        ],
+        "pricing": {
+          "hour": 0.05
+        }
+      },
+      "userdata": "",
+      "root_password": "soaz0#a=cd",
+      "href": "/devices/5bd27259-8b71-48f7-879d-c4e695cb2f31"
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=1"
+    },
+    "previous": {
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=1"
+    },
+    "self": {
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=2"
+    },
+    "next": null,
+    "last": {
+      "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54/devices?page=2"
+    },
+    "total": 7
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/devices.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/devices.json b/packet/src/test/resources/devices.json
new file mode 100644
index 0000000..b9fbade
--- /dev/null
+++ b/packet/src/test/resources/devices.json
@@ -0,0 +1,282 @@
+{
+  "devices": [
+    {
+      "id": "98e22032-579e-4c04-bb12-05cc6a3864c8",
+      "short_id": "98e22032",
+      "hostname": "test",
+      "description": null,
+      "state": "provisioning",
+      "tags": [],
+      "billing_cycle": "hourly",
+      "user": "root",
+      "iqn": "iqn.2017-01.net.packet:device.98e22032",
+      "locked": false,
+      "bonding_mode": 5,
+      "created_at": "2017-01-03T09:47:59Z",
+      "updated_at": "2017-01-03T09:50:13Z",
+      "provisioning_percentage": 50,
+      "operating_system": {
+        "id": "68bad60d-f5a7-45c2-ad09-573edaad3a3c",
+        "slug": "centos_7",
+        "name": "Centos 7",
+        "distro": "centos",
+        "version": "7",
+        "provisionable_on": [
+          "baremetal_0",
+          "baremetal_1",
+          "baremetal_2",
+          "baremetal_3"
+        ]
+      },
+      "facility": {
+        "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+        "name": "Parsippany, NJ",
+        "code": "ewr1",
+        "features": [
+          "baremetal",
+          "storage"
+        ],
+        "address": null
+      },
+      "project": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "ssh_keys": [
+        {
+          "href": "/ssh-keys/084a5dec-30be-415a-8937-9c615932e459"
+        },
+        {
+          "href": "/ssh-keys/a8d6cc17-7d9d-4fb9-8190-afdb301b67df"
+        },
+        {
+          "href": "/ssh-keys/a3d8bebe-574f-427d-80ee-bc2ba17f7074"
+        },
+        {
+          "href": "/ssh-keys/eacfb002-45e1-4047-a0d5-cd9d8bab19ed"
+        },
+        {
+          "href": "/ssh-keys/bba63e41-b12c-493a-81d4-e52f50f247ed"
+        }
+      ],
+      "project_lite": {
+        "href": "/projects/93907f48-adfe-43ed-ad89-0e6e83721a54"
+      },
+      "volumes": [],
+      "ip_addresses": [
+        {
+          "id": "5d0262c7-1727-411c-94c4-9e6f15490dd3",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-03T09:47:59Z",
+          "public": true,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "assigned_to": {
+            "href": "/devices/98e22032-579e-4c04-bb12-05cc6a3864c8"
+          },
+          "network": "147.75.106.80",
+          "address": "147.75.106.81",
+          "gateway": "147.75.106.80",
+          "href": "/ips/5d0262c7-1727-411c-94c4-9e6f15490dd3"
+        },
+        {
+          "id": "f7d0e65c-eb3b-42bd-af9b-ad3a736d8d43",
+          "address_family": 6,
+          "netmask": "ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe",
+          "created_at": "2017-01-03T09:47:59Z",
+          "public": true,
+          "cidr": 127,
+          "management": true,
+          "manageable": true,
+          "assigned_to": {
+            "href": "/devices/98e22032-579e-4c04-bb12-05cc6a3864c8"
+          },
+          "network": "2604:1380:2:9800::",
+          "address": "2604:1380:2:9800::1",
+          "gateway": "2604:1380:2:9800::",
+          "href": "/ips/f7d0e65c-eb3b-42bd-af9b-ad3a736d8d43"
+        },
+        {
+          "id": "a3d00b4e-d74f-4ac2-8bc9-91065d815b41",
+          "address_family": 4,
+          "netmask": "255.255.255.254",
+          "created_at": "2017-01-03T09:47:59Z",
+          "public": false,
+          "cidr": 31,
+          "management": true,
+          "manageable": true,
+          "assigned_to": {
+            "href": "/devices/98e22032-579e-4c04-bb12-05cc6a3864c8"
+          },
+          "network": "10.99.214.0",
+          "address": "10.99.214.1",
+          "gateway": "10.99.214.0",
+          "href": "/ips/a3d00b4e-d74f-4ac2-8bc9-91065d815b41"
+        }
+      ],
+      "provisioning_events": [
+        {
+          "id": "bd62123b-afed-4e54-b1b9-89e219ba9cf0",
+          "type": "provisioning.101",
+          "body": "Provisioning started",
+          "created_at": "2017-01-03T09:47:59Z",
+          "relationships": [
+            {
+              "href": "#81909921-255e-413c-883a-c58d14c801ae"
+            }
+          ],
+          "interpolated": "Provisioning started",
+          "href": "/events/bd62123b-afed-4e54-b1b9-89e219ba9cf0"
+        },
+        {
+          "id": "14b4a9e6-be90-40ee-be48-b272f855e39c",
+          "type": "provisioning.102",
+          "body": "Network configured with addresses 147.75.106.81, 2604:1380:2:9800::1, and 10.99.214.1",
+          "created_at": "2017-01-03T09:48:46Z",
+          "relationships": [
+            {
+              "href": "#28588657-b8bf-44a1-98d3-27f8e04b660a"
+            }
+          ],
+          "interpolated": "Network configured with addresses 147.75.106.81, 2604:1380:2:9800::1, and 10.99.214.1",
+          "href": "/events/14b4a9e6-be90-40ee-be48-b272f855e39c"
+        },
+        {
+          "id": "57e3cadb-f9aa-4c73-be54-3c83e6cf462e",
+          "type": "provisioning.103",
+          "body": "Configuration written, restarting device",
+          "created_at": "2017-01-03T09:48:57Z",
+          "relationships": [
+            {
+              "href": "#25a7e807-f4aa-4f36-83cd-8347baeb26bc"
+            }
+          ],
+          "interpolated": "Configuration written, restarting device",
+          "href": "/events/57e3cadb-f9aa-4c73-be54-3c83e6cf462e"
+        },
+        {
+          "id": "b8322996-f57e-4c87-96cc-a16f33a0c305",
+          "type": "provisioning.104",
+          "body": "Connected to magic install system",
+          "created_at": "2017-01-03T09:50:13Z",
+          "relationships": [
+            {
+              "href": "#a4825d20-f7f2-426a-88db-38696bd3dfd6"
+            }
+          ],
+          "interpolated": "Connected to magic install system",
+          "href": "/events/b8322996-f57e-4c87-96cc-a16f33a0c305"
+        },
+        {
+          "id": "f00a00a9-7c59-420d-bdcf-2c0993303cf6",
+          "type": "provisioning.105",
+          "body": "Server partitions created",
+          "created_at": "2017-01-03T09:50:13Z",
+          "relationships": [
+            {
+              "href": "#6140659b-5e3d-4686-be8f-8879b6d3e27f"
+            }
+          ],
+          "interpolated": "Server partitions created",
+          "href": "/events/f00a00a9-7c59-420d-bdcf-2c0993303cf6"
+        },
+        {
+          "id": null,
+          "type": "provisioning.106",
+          "body": "Operating system packages installed",
+          "created_at": null,
+          "relationships": [],
+          "interpolated": "Operating system packages installed"
+        },
+        {
+          "id": null,
+          "type": "provisioning.107",
+          "body": "Server networking interfaces configured",
+          "created_at": null,
+          "relationships": [],
+          "interpolated": "Server networking interfaces configured"
+        },
+        {
+          "id": null,
+          "type": "provisioning.108",
+          "body": "Cloud-init packages installed and configured",
+          "created_at": null,
+          "relationships": [],
+          "interpolated": "Cloud-init packages installed and configured"
+        },
+        {
+          "id": null,
+          "type": "provisioning.109",
+          "body": "Installation finished, rebooting server",
+          "created_at": null,
+          "relationships": [],
+          "interpolated": "Installation finished, rebooting server"
+        },
+        {
+          "id": null,
+          "type": "provisioning.109",
+          "body": "Installation finished, rebooting server",
+          "created_at": null,
+          "relationships": [],
+          "interpolated": "Installation finished, rebooting server"
+        }
+      ],
+      "plan": {
+        "id": "e69c0169-4726-46ea-98f1-939c9e8a3607",
+        "slug": "baremetal_0",
+        "name": "Type 0",
+        "description": "Our Type 0 configuration is a general use \"cloud killer\" server, with a Intel Atom 2.4Ghz processor and 8GB of RAM.",
+        "line": "baremetal",
+        "specs": {
+          "cpus": [
+            {
+              "count": 1,
+              "type": "Intel Atom C2550 @ 2.4Ghz"
+            }
+          ],
+          "memory": {
+            "total": "8GB"
+          },
+          "drives": [
+            {
+              "count": 1,
+              "size": "80GB",
+              "type": "SSD"
+            }
+          ],
+          "nics": [
+            {
+              "count": 2,
+              "type": "1Gbps"
+            }
+          ],
+          "features": {
+            "raid": false,
+            "txt": true
+          }
+        },
+        "available_in": [
+          {
+            "href": "/facilities/2b70eb8f-fa18-47c0-aba7-222a842362fd"
+          },
+          {
+            "href": "/facilities/8e6470b3-b75e-47d1-bb93-45b225750975"
+          },
+          {
+            "href": "/facilities/8ea03255-89f9-4e62-9d3f-8817db82ceed"
+          },
+          {
+            "href": "/facilities/e1e9c52e-a0bc-4117-b996-0fc94843ea09"
+          }
+        ],
+        "pricing": {
+          "hour": 0.05
+        }
+      },
+      "userdata": "",
+      "root_password": ",q4*a8(eny",
+      "href": "/devices/98e22032-579e-4c04-bb12-05cc6a3864c8"
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/facilities-first.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/facilities-first.json b/packet/src/test/resources/facilities-first.json
new file mode 100644
index 0000000..5decf9a
--- /dev/null
+++ b/packet/src/test/resources/facilities-first.json
@@ -0,0 +1,39 @@
+{
+  "facilities": [
+    {
+      "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+      "name": "Parsippany, NJ",
+      "code": "ewr1",
+      "features": [
+        "baremetal",
+        "storage"
+      ],
+      "address": null
+    },
+    {
+      "id": "8e6470b3-b75e-47d1-bb93-45b225750975",
+      "name": "Amsterdam, NL",
+      "code": "ams1",
+      "features": [
+        "storage"
+      ],
+      "address": null
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/facilities?page=1"
+    },
+    "previous": null,
+    "self": {
+      "href": "/facilities?page=1"
+    },
+    "next": {
+      "href": "/facilities?page=2"
+    },
+    "last": {
+      "href": "/facilities?page=2"
+    },
+    "total": 3
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/facilities-last.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/facilities-last.json b/packet/src/test/resources/facilities-last.json
new file mode 100644
index 0000000..f5e6d1c
--- /dev/null
+++ b/packet/src/test/resources/facilities-last.json
@@ -0,0 +1,27 @@
+{
+  "facilities": [
+    {
+      "id": "2b70eb8f-fa18-47c0-aba7-222a842362fd",
+      "name": "Sunnyvale, CA",
+      "code": "sjc1",
+      "features": [],
+      "address": null
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/facilities?page=1"
+    },
+    "previous": {
+      "href": "/facilities?page=1"
+    },
+    "self": {
+      "href": "/facilities?page=2"
+    },
+    "next": null,
+    "last": {
+      "href": "/facilities?page=2"
+    },
+    "total": 3
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/facilities.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/facilities.json b/packet/src/test/resources/facilities.json
new file mode 100644
index 0000000..ed6ab4f
--- /dev/null
+++ b/packet/src/test/resources/facilities.json
@@ -0,0 +1,30 @@
+{
+  "facilities": [
+    {
+      "id": "e1e9c52e-a0bc-4117-b996-0fc94843ea09",
+      "name": "Parsippany, NJ",
+      "code": "ewr1",
+      "features": [
+        "baremetal",
+        "storage"
+      ],
+      "address": null
+    },
+    {
+      "id": "8e6470b3-b75e-47d1-bb93-45b225750975",
+      "name": "Amsterdam, NL",
+      "code": "ams1",
+      "features": [
+        "storage"
+      ],
+      "address": null
+    },
+    {
+      "id": "2b70eb8f-fa18-47c0-aba7-222a842362fd",
+      "name": "Sunnyvale, CA",
+      "code": "sjc1",
+      "features": [],
+      "address": null
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/operatingSystems-first.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/operatingSystems-first.json b/packet/src/test/resources/operatingSystems-first.json
new file mode 100644
index 0000000..1438108
--- /dev/null
+++ b/packet/src/test/resources/operatingSystems-first.json
@@ -0,0 +1,96 @@
+{
+  "operating_systems": [
+    {
+      "id": "06e21644-a769-11e6-80f5-76304dec7eb7",
+      "slug": "alpine_3",
+      "name": "Alpine 3",
+      "distro": "alpine",
+      "version": "3",
+      "provisionable_on": []
+    },
+    {
+      "id": "06e21978-a769-11e6-80f5-76304dec7eb7",
+      "slug": "centos_6",
+      "name": "CentOS 6",
+      "distro": "centos",
+      "version": "6",
+      "provisionable_on": []
+    },
+    {
+      "id": "68bad60d-f5a7-45c2-ad09-573edaad3a3c",
+      "slug": "centos_7",
+      "name": "Centos 7",
+      "distro": "centos",
+      "version": "7",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e21ce8-a769-11e6-80f5-76304dec7eb7",
+      "slug": "coreos_alpha",
+      "name": "CoreOS Alpha",
+      "distro": "coreos",
+      "version": "alpha",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "6345f310-0bb0-4b59-b051-954fed05183a",
+      "slug": "coreos_beta",
+      "name": "CoreOS Beta",
+      "distro": "coreos",
+      "version": "beta",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "d61c3912-8422-4daf-835e-854efa0062e4",
+      "slug": "coreos_stable",
+      "name": "CoreOS Stable",
+      "distro": "coreos",
+      "version": "stable",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e21e78-a769-11e6-80f5-76304dec7eb7",
+      "slug": "debian_7",
+      "name": "Debian 7",
+      "distro": "debian",
+      "version": "7",
+      "provisionable_on": []
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/operating-systems?page=1"
+    },
+    "previous": null,
+    "self": {
+      "href": "/operating-systems?page=1"
+    },
+    "next": {
+      "href": "/operating-systems?page=2"
+    },
+    "last": {
+      "href": "/operating-systems?page=2"
+    },
+    "total": 14
+  }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/a812591f/packet/src/test/resources/operatingSystems-last.json
----------------------------------------------------------------------
diff --git a/packet/src/test/resources/operatingSystems-last.json b/packet/src/test/resources/operatingSystems-last.json
new file mode 100644
index 0000000..2c3c8e3
--- /dev/null
+++ b/packet/src/test/resources/operatingSystems-last.json
@@ -0,0 +1,106 @@
+{
+  "operating_systems": [
+    {
+      "id": "88239019-abc7-41e8-9a4d-cd334da97ff1",
+      "slug": "debian_8",
+      "name": "Debian 8",
+      "distro": "debian",
+      "version": "8",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e21ffe-a769-11e6-80f5-76304dec7eb7",
+      "slug": "deprovision",
+      "name": "Deprovision",
+      "distro": "centos",
+      "version": "",
+      "provisionable_on": []
+    },
+    {
+      "id": "06e22198-a769-11e6-80f5-76304dec7eb7",
+      "slug": "freebsd_10_3",
+      "name": "FreeBSD 10.3",
+      "distro": "freebsd",
+      "version": "10.3",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e22328-a769-11e6-80f5-76304dec7eb7",
+      "slug": "rancher",
+      "name": "RancherOS",
+      "distro": "rancher",
+      "version": "latest",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e224b8-a769-11e6-80f5-76304dec7eb7",
+      "slug": "ubuntu_14_04",
+      "name": "Ubuntu 14.04 LTS",
+      "distro": "ubuntu",
+      "version": "14.04",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "1b9b78e3-de68-466e-ba00-f2123e89c112",
+      "slug": "ubuntu_16_04",
+      "name": "Ubuntu 16.04 LTS",
+      "distro": "ubuntu",
+      "version": "16.04",
+      "provisionable_on": [
+        "baremetal_0",
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_2a",
+        "baremetal_3"
+      ]
+    },
+    {
+      "id": "06e22972-a769-11e6-80f5-76304dec7eb7",
+      "slug": "windows_2012_rc2",
+      "name": "Windows 2012 RC2",
+      "distro": "windows",
+      "version": "2012 RC2",
+      "provisionable_on": [
+        "baremetal_1",
+        "baremetal_2",
+        "baremetal_3"
+      ]
+    }
+  ],
+  "meta": {
+    "first": {
+      "href": "/operating-systems?page=1"
+    },
+    "previous": {
+      "href": "/operating-systems?page=1"
+    },
+    "self": {
+      "href": "/operating-systems?page=2"
+    },
+    "next": null,
+    "last": {
+      "href": "/operating-systems?page=2"
+    },
+    "total": 14
+  }
+}