You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dlab.apache.org by bh...@apache.org on 2019/03/12 16:28:10 UTC

[incubator-dlab] branch billing_gcp created (now 5a9566f)

This is an automated email from the ASF dual-hosted git repository.

bhliva pushed a change to branch billing_gcp
in repository https://gitbox.apache.org/repos/asf/incubator-dlab.git.


      at 5a9566f  DLAB-23 added gcp billing + refactored current billing implementation

This branch includes the following new commits:

     new 310b9c5  DLAB-000 billing mock on gcp implemented
     new 5a9566f  DLAB-23 added gcp billing + refactored current billing implementation

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@dlab.apache.org
For additional commands, e-mail: commits-help@dlab.apache.org


[incubator-dlab] 02/02: DLAB-23 added gcp billing + refactored current billing implementation

Posted by bh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bhliva pushed a commit to branch billing_gcp
in repository https://gitbox.apache.org/repos/asf/incubator-dlab.git

commit 5a9566fba0752dd754661df633dcea15cce3fc3a
Author: bhliva <bo...@epam.com>
AuthorDate: Tue Mar 12 16:03:13 2019 +0200

    DLAB-23 added gcp billing + refactored current billing implementation
---
 .../epam/dlab/backendapi/dao/BaseBillingDAO.java   | 655 +++++++++++++--------
 .../dlab/backendapi/dao/aws/AwsBillingDAO.java     | 191 ++----
 .../dlab/backendapi/dao/azure/AzureBillingDAO.java | 205 ++-----
 .../dlab/backendapi/dao/gcp/GcpBillingDao.java     | 146 ++---
 .../backendapi/resources/dto/BillingFilter.java    |   4 +-
 .../resources/dto/aws/AwsBillingFilter.java        |   6 +
 .../resources/dto/azure/AzureBillingFilter.java    |  12 +-
 .../resources/dto/gcp/GcpBillingFilter.java        |   5 +
 .../webapp/src/dictionary/azure.dictionary.ts      |   4 +-
 .../webapp/src/dictionary/gcp.dictionary.ts        |   8 +-
 10 files changed, 576 insertions(+), 660 deletions(-)

diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/BaseBillingDAO.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/BaseBillingDAO.java
index 297f561..11b45ed 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/BaseBillingDAO.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/BaseBillingDAO.java
@@ -16,13 +16,22 @@
 
 package com.epam.dlab.backendapi.dao;
 
+import com.epam.dlab.MongoKeyWords;
 import com.epam.dlab.auth.UserInfo;
 import com.epam.dlab.backendapi.resources.dto.BillingFilter;
+import com.epam.dlab.backendapi.roles.RoleType;
+import com.epam.dlab.backendapi.roles.UserRoles;
+import com.epam.dlab.billing.BillingCalculationUtils;
+import com.epam.dlab.billing.DlabResourceType;
 import com.epam.dlab.dto.UserInstanceStatus;
 import com.epam.dlab.dto.base.DataEngineType;
+import com.epam.dlab.model.aws.ReportLine;
 import com.google.common.collect.Lists;
 import com.google.inject.Inject;
+import com.mongodb.client.AggregateIterable;
 import com.mongodb.client.FindIterable;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Filters;
 import lombok.Getter;
 import lombok.ToString;
 import lombok.extern.slf4j.Slf4j;
@@ -37,250 +46,422 @@ import java.util.function.Supplier;
 import static com.epam.dlab.backendapi.dao.ComputationalDAO.COMPUTATIONAL_ID;
 import static com.epam.dlab.backendapi.dao.ExploratoryDAO.COMPUTATIONAL_RESOURCES;
 import static com.epam.dlab.backendapi.dao.ExploratoryDAO.EXPLORATORY_ID;
-import static com.epam.dlab.backendapi.dao.MongoCollections.BILLING;
-import static com.epam.dlab.backendapi.dao.MongoCollections.USER_INSTANCES;
+import static com.epam.dlab.backendapi.dao.MongoCollections.*;
+import static com.epam.dlab.model.aws.ReportLine.FIELD_RESOURCE_TYPE;
+import static com.epam.dlab.model.aws.ReportLine.FIELD_USAGE_DATE;
 import static com.mongodb.client.model.Accumulators.sum;
 import static com.mongodb.client.model.Aggregates.group;
 import static com.mongodb.client.model.Aggregates.match;
-import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.*;
 import static com.mongodb.client.model.Projections.*;
 import static java.util.Collections.singletonList;
 
 @Slf4j
 public abstract class BaseBillingDAO<T extends BillingFilter> extends BaseDAO implements BillingDAO<T> {
 
-	public static final String SHAPE = "shape";
-	public static final String SERVICE_BASE_NAME = "service_base_name";
-	public static final String ITEMS = "lines";
-	public static final String COST_TOTAL = "cost_total";
-	public static final String FULL_REPORT = "full_report";
-
-	private static final String MASTER_NODE_SHAPE = "master_node_shape";
-	private static final String SLAVE_NODE_SHAPE = "slave_node_shape";
-	private static final String TOTAL_INSTANCE_NUMBER = "total_instance_number";
-
-	private static final String DATAENGINE_SHAPE = "dataengine_instance_shape";
-	private static final String DATAENGINE_INSTANCE_COUNT = "dataengine_instance_count";
-
-	private static final String DATAENGINE_DOCKER_IMAGE = "image";
-	private static final int ONE_HUNDRED = 100;
-	private static final String TOTAL_FIELD_NAME = "total";
-	private static final String COST_FIELD = "$cost";
-	public static final String SHARED_RESOURCE_NAME = "Shared resource";
-
-	@Inject
-	protected SettingsDAO settings;
-	@Inject
-	private UserSettingsDAO userSettingsDAO;
-
-	protected Map<String, ShapeInfo> getShapes(List<String> shapeNames) {
-		FindIterable<Document> userInstances = getUserInstances();
-		final Map<String, ShapeInfo> shapes = new HashMap<>();
-
-		for (Document d : userInstances) {
-			getExploratoryShape(shapeNames, d)
-					.ifPresent(shapeInfo -> shapes.put(d.getString(EXPLORATORY_ID), shapeInfo));
-			@SuppressWarnings("unchecked")
-			List<Document> comp = (List<Document>) d.get(COMPUTATIONAL_RESOURCES);
-			comp.forEach(computational ->
-					getComputationalShape(shapeNames, computational)
-							.ifPresent(shapeInfo -> shapes.put(computational.getString(COMPUTATIONAL_ID), shapeInfo)));
-		}
-
-		appendSsnAndEdgeNodeType(shapeNames, shapes);
-
-		log.trace("Loaded shapes is {}", shapes);
-		return shapes;
-	}
-
-	@Override
-	public Double getTotalCost() {
-		return aggregateBillingData(singletonList(group(null, sum(TOTAL_FIELD_NAME, COST_FIELD))));
-	}
-
-	@Override
-	public Double getUserCost(String user) {
-		final List<Bson> pipeline = Arrays.asList(match(eq(USER, user)),
-				group(null, sum(TOTAL_FIELD_NAME, COST_FIELD)));
-		return aggregateBillingData(pipeline);
-	}
-
-	@Override
-	public int getBillingQuoteUsed() {
-		return toPercentage(() -> settings.getMaxBudget(), getTotalCost());
-	}
-
-	@Override
-	public int getBillingUserQuoteUsed(String user) {
-		return toPercentage(() -> userSettingsDAO.getAllowedBudget(user), getUserCost(user));
-	}
-
-	@Override
-	public boolean isBillingQuoteReached() {
-		return getBillingQuoteUsed() >= ONE_HUNDRED;
-	}
-
-	@Override
-	public boolean isUserQuoteReached(String user) {
-		final Double userCost = getUserCost(user);
-		return userSettingsDAO.getAllowedBudget(user)
-				.filter(allowedBudget -> userCost.intValue() != 0 && allowedBudget <= userCost)
-				.isPresent();
-	}
-
-	protected String getUserOrDefault(String user) {
-		return StringUtils.isNotBlank(user) ? user : SHARED_RESOURCE_NAME;
-	}
-
-	private Integer toPercentage(Supplier<Optional<Integer>> allowedBudget, Double totalCost) {
-		return allowedBudget.get()
-				.map(userBudget -> (totalCost * ONE_HUNDRED) / userBudget)
-				.map(Double::intValue)
-				.orElse(BigDecimal.ZERO.intValue());
-	}
-
-
-	private Optional<ShapeInfo> getComputationalShape(List<String> shapeNames, Document c) {
-		return isDataEngine(c.getString(DATAENGINE_DOCKER_IMAGE)) ? getDataEngineShape(shapeNames, c) :
-				getDataEngineServiceShape(shapeNames, c);
-	}
-
-	private Double aggregateBillingData(List<Bson> pipeline) {
-		return Optional.ofNullable(aggregate(BILLING, pipeline).first())
-				.map(d -> d.getDouble(TOTAL_FIELD_NAME))
-				.orElse(BigDecimal.ZERO.doubleValue());
-	}
-
-	private FindIterable<Document> getUserInstances() {
-		return getCollection(USER_INSTANCES)
-				.find()
-				.projection(
-						fields(excludeId(),
-								include(SHAPE, EXPLORATORY_ID, STATUS,
-										COMPUTATIONAL_RESOURCES + "." + COMPUTATIONAL_ID,
-										COMPUTATIONAL_RESOURCES + "." + MASTER_NODE_SHAPE,
-										COMPUTATIONAL_RESOURCES + "." + SLAVE_NODE_SHAPE,
-										COMPUTATIONAL_RESOURCES + "." + TOTAL_INSTANCE_NUMBER,
-										COMPUTATIONAL_RESOURCES + "." + DATAENGINE_SHAPE,
-										COMPUTATIONAL_RESOURCES + "." + DATAENGINE_INSTANCE_COUNT,
-										COMPUTATIONAL_RESOURCES + "." + DATAENGINE_DOCKER_IMAGE,
-										COMPUTATIONAL_RESOURCES + "." + STATUS
-								)));
-	}
-
-	private Optional<ShapeInfo> getExploratoryShape(List<String> shapeNames, Document d) {
-		final String shape = d.getString(SHAPE);
-		if (isShapeAcceptable(shapeNames, shape)) {
-			return Optional.of(new ShapeInfo(shape, UserInstanceStatus.of(d.getString(STATUS))));
-		}
-		return Optional.empty();
-	}
-
-	private boolean isDataEngine(String dockerImage) {
-		return DataEngineType.fromDockerImageName(dockerImage) == DataEngineType.SPARK_STANDALONE;
-	}
-
-	private Optional<ShapeInfo> getDataEngineServiceShape(List<String> shapeNames,
-														  Document c) {
-		final String desMasterShape = c.getString(MASTER_NODE_SHAPE);
-		final String desSlaveShape = c.getString(SLAVE_NODE_SHAPE);
-		if (isShapeAcceptable(shapeNames, desMasterShape, desSlaveShape)) {
-			return Optional.of(new ShapeInfo(desMasterShape, desSlaveShape, c.getString(TOTAL_INSTANCE_NUMBER),
-					UserInstanceStatus.of(c.getString(STATUS))));
-		}
-		return Optional.empty();
-	}
-
-	private Optional<ShapeInfo> getDataEngineShape(List<String> shapeNames, Document c) {
-		final String dataEngineShape = c.getString(DATAENGINE_SHAPE);
-		if ((isShapeAcceptable(shapeNames, dataEngineShape))
-				&& StringUtils.isNotEmpty(c.getString(COMPUTATIONAL_ID))) {
-
-			return Optional.of(new ShapeInfo(dataEngineShape, c.getString(DATAENGINE_INSTANCE_COUNT),
-					UserInstanceStatus.of(c.getString(STATUS))));
-		}
-		return Optional.empty();
-	}
-
-	private boolean isShapeAcceptable(List<String> shapeNames, String... shapes) {
-		return shapeNames == null || shapeNames.isEmpty() || Arrays.stream(shapes).anyMatch(shapeNames::contains);
-	}
-
-	protected abstract void appendSsnAndEdgeNodeType(List<String> shapeNames, Map<String, ShapeInfo> shapes);
-
-	protected String generateShapeName(ShapeInfo shape) {
-		return Optional.ofNullable(shape).map(ShapeInfo::getName).orElse(StringUtils.EMPTY);
-	}
-
-	protected void usersToLowerCase(List<String> users) {
-		if (users != null) {
-			users.replaceAll(u -> u != null ? u.toLowerCase() : null);
-		}
-	}
-
-	protected void setUserFilter(UserInfo userInfo, BillingFilter filter, boolean isFullReport) {
-		if (isFullReport) {
-			usersToLowerCase(filter.getUser());
-		} else {
-			filter.setUser(Lists.newArrayList(userInfo.getName().toLowerCase()));
-		}
-	}
-
-	/**
-	 * Store shape info
-	 */
-	@Getter
-	@ToString
-	protected class ShapeInfo {
-		private static final String DES_NAME_FORMAT = "Master: %s%sSlave:  %d x %s";
-		private static final String DE_NAME_FORMAT = "%d x %s";
-		private final boolean isDataEngine;
-		private final String shape;
-		private final String slaveShape;
-		private final String slaveCount;
-		private final boolean isExploratory;
-		private final UserInstanceStatus status;
-
-		private ShapeInfo(boolean isDataEngine, String shape, String slaveShape, String slaveCount, boolean
-				isExploratory, UserInstanceStatus status) {
-			this.isDataEngine = isDataEngine;
-			this.shape = shape;
-			this.slaveShape = slaveShape;
-			this.slaveCount = slaveCount;
-			this.isExploratory = isExploratory;
-			this.status = status;
-		}
-
-		public ShapeInfo(String shape, UserInstanceStatus status) {
-			this(false, shape, null, null, true, status);
-		}
-
-		ShapeInfo(String shape, String slaveShape, String slaveCount, UserInstanceStatus status) {
-			this(false, shape, slaveShape, slaveCount, false, status);
-		}
-
-
-		ShapeInfo(String shape, String slaveCount, UserInstanceStatus status) {
-			this(true, shape, null, slaveCount, false, status);
-		}
-
-		public String getName() {
-			if (isExploratory) {
-				return shape;
-			} else {
-				return clusterName();
-			}
-		}
-
-		private String clusterName() {
-			try {
-				final Integer count = Integer.valueOf(slaveCount);
-				return isDataEngine ? String.format(DE_NAME_FORMAT, count, shape) :
-						String.format(DES_NAME_FORMAT, shape, System.lineSeparator(), count - 1, slaveShape);
-			} catch (NumberFormatException e) {
-				log.error("Cannot parse string {} to integer", slaveCount);
-				return StringUtils.EMPTY;
-			}
-		}
-	}
+    public static final String SHAPE = "shape";
+    public static final String SERVICE_BASE_NAME = "service_base_name";
+    public static final String ITEMS = "lines";
+    public static final String COST_TOTAL = "cost_total";
+    public static final String FULL_REPORT = "full_report";
+
+    private static final String MASTER_NODE_SHAPE = "master_node_shape";
+    private static final String SLAVE_NODE_SHAPE = "slave_node_shape";
+    private static final String TOTAL_INSTANCE_NUMBER = "total_instance_number";
+
+    private static final String DATAENGINE_SHAPE = "dataengine_instance_shape";
+    private static final String DATAENGINE_INSTANCE_COUNT = "dataengine_instance_count";
+
+    private static final String DATAENGINE_DOCKER_IMAGE = "image";
+    private static final int ONE_HUNDRED = 100;
+    private static final String TOTAL_FIELD_NAME = "total";
+    private static final String COST_FIELD = "$cost";
+    public static final String SHARED_RESOURCE_NAME = "Shared resource";
+
+    @Inject
+    protected SettingsDAO settings;
+    @Inject
+    private UserSettingsDAO userSettingsDAO;
+
+    @Override
+    public Document getReport(UserInfo userInfo, T filter) {
+        boolean isFullReport = UserRoles.checkAccess(userInfo, RoleType.PAGE, "/api/infrastructure_provision/billing");
+        setUserFilter(userInfo, filter, isFullReport);
+        List<Bson> matchCriteria = matchCriteria(filter);
+        List<Bson> pipeline = new ArrayList<>();
+        if (!matchCriteria.isEmpty()) {
+            pipeline.add(Aggregates.match(Filters.and(matchCriteria)));
+        }
+        pipeline.add(groupCriteria());
+        pipeline.add(sortCriteria());
+        final Map<String, ShapeInfo> shapes = getShapes(filter.getShapes());
+        return prepareReport(filter.getStatuses(), !filter.getShapes().isEmpty(), getCollection(BILLING).aggregate(pipeline), shapes, isFullReport); //TODO add shapes
+    }
+
+    private Document prepareReport(List<UserInstanceStatus> statuses, boolean filterByShape,
+                                   AggregateIterable<Document> agg,
+                                   Map<String, ShapeInfo> shapes, boolean fullReport) {
+
+        List<Document> reportItems = new ArrayList<>();
+
+        Date usageDateStart = null;
+        Date usageDateEnd = null;
+        double costTotal = 0D;
+
+        for (Document d : agg) {
+            Document id = (Document) d.get(MongoKeyWords.MONGO_ID);
+            String resourceId = id.getString(dlabIdFieldName());
+            ShapeInfo shape = shapes.get(resourceId);
+            final UserInstanceStatus status = Optional.ofNullable(shape).map(ShapeInfo::getStatus).orElse(null);
+            if ((filterByShape && shape == null) ||
+                    (!statuses.isEmpty() && statuses.stream().noneMatch(s -> s.equals(status)))) {
+                continue;
+            }
+
+
+            final Date dateStart = d.getDate(usageDateFromFieldName());
+            if (usageDateStart == null || dateStart.before(usageDateStart)) {
+                usageDateStart = dateStart;
+            }
+            Date dateEnd = d.getDate(usageDateToFieldName());
+            if (usageDateEnd == null || dateEnd.after(usageDateEnd)) {
+                usageDateEnd = dateEnd;
+            }
+
+            costTotal += d.getDouble(MongoKeyWords.COST);
+
+            final String statusString = Optional
+                    .ofNullable(status)
+                    .map(UserInstanceStatus::toString)
+                    .orElse(StringUtils.EMPTY);
+            Document item = new Document()
+                    .append(MongoKeyWords.DLAB_USER, getUserOrDefault(id.getString(USER)))
+                    .append(dlabIdFieldName(), resourceId)
+                    .append(shapeFieldName(), generateShapeName(shape))
+                    .append(FIELD_RESOURCE_TYPE, DlabResourceType.getResourceTypeName(id.getString("dlab_resource_type"))) //todo check on azure!!!
+                    .append(STATUS,
+                            statusString)
+                    .append(productFieldName(), id.getString(productFieldName()))
+                    .append(MongoKeyWords.COST, d.getDouble(MongoKeyWords.COST))
+                    .append(costFieldName(), BillingCalculationUtils.formatDouble(d.getDouble(MongoKeyWords
+                            .COST)))
+                    .append(currencyCodeFieldName(), id.getString(currencyCodeFieldName()))
+                    .append(usageDateFromFieldName(), dateStart)
+                    .append(usageDateToFieldName(), dateEnd);
+
+
+            reportItems.add(item);
+        }
+
+        return new Document()
+                .append(SERVICE_BASE_NAME, settings.getServiceBaseName())
+                .append(usageDateFromFieldName(), usageDateStart)
+                .append(usageDateToFieldName(), usageDateEnd)
+                .append(ITEMS, reportItems)
+                .append(COST_TOTAL, BillingCalculationUtils.formatDouble(BillingCalculationUtils.round
+                        (costTotal, 2)))
+                .append(currencyCodeFieldName(), (reportItems.isEmpty() ? null :
+                        reportItems.get(0).getString(currencyCodeFieldName())))
+                .append(FULL_REPORT, fullReport);
+
+    }
+
+    protected String currencyCodeFieldName() {
+        return "currency_code";
+    }
+
+    protected String usageDateToFieldName() {
+        return MongoKeyWords.USAGE_TO;
+    }
+
+    protected String costFieldName() {
+        return MongoKeyWords.COST;
+    }
+
+    protected String productFieldName() {
+        return ReportLine.FIELD_PRODUCT;
+    }
+
+    protected String usageDateFromFieldName() {
+        return MongoKeyWords.USAGE_FROM;
+    }
+
+    protected String dlabIdFieldName() {
+        return ReportLine.FIELD_DLAB_ID;
+    }
+
+    protected String shapeFieldName() {
+        return SHAPE;
+    }
+
+    protected abstract Bson sortCriteria();
+
+    protected abstract Bson groupCriteria();
+
+    protected Map<String, ShapeInfo> getShapes(List<String> shapeNames) {
+        FindIterable<Document> userInstances = getUserInstances();
+        final Map<String, ShapeInfo> shapes = new HashMap<>();
+
+        for (Document d : userInstances) {
+            getExploratoryShape(shapeNames, d)
+                    .ifPresent(shapeInfo -> shapes.put(d.getString(EXPLORATORY_ID), shapeInfo));
+            @SuppressWarnings("unchecked")
+            List<Document> comp = (List<Document>) d.get(COMPUTATIONAL_RESOURCES);
+            comp.forEach(computational ->
+                    getComputationalShape(shapeNames, computational)
+                            .ifPresent(shapeInfo -> shapes.put(computational.getString(COMPUTATIONAL_ID), shapeInfo)));
+        }
+
+        appendSsnAndEdgeNodeType(shapeNames, shapes);
+
+        log.trace("Loaded shapes is {}", shapes);
+        return shapes;
+    }
+
+    @Override
+    public Double getTotalCost() {
+        return aggregateBillingData(singletonList(group(null, sum(TOTAL_FIELD_NAME, COST_FIELD))));
+    }
+
+    @Override
+    public Double getUserCost(String user) {
+        final List<Bson> pipeline = Arrays.asList(match(eq(USER, user)),
+                group(null, sum(TOTAL_FIELD_NAME, COST_FIELD)));
+        return aggregateBillingData(pipeline);
+    }
+
+    @Override
+    public int getBillingQuoteUsed() {
+        return toPercentage(() -> settings.getMaxBudget(), getTotalCost());
+    }
+
+    @Override
+    public int getBillingUserQuoteUsed(String user) {
+        return toPercentage(() -> userSettingsDAO.getAllowedBudget(user), getUserCost(user));
+    }
+
+    @Override
+    public boolean isBillingQuoteReached() {
+        return getBillingQuoteUsed() >= ONE_HUNDRED;
+    }
+
+    @Override
+    public boolean isUserQuoteReached(String user) {
+        final Double userCost = getUserCost(user);
+        return userSettingsDAO.getAllowedBudget(user)
+                .filter(allowedBudget -> userCost.intValue() != 0 && allowedBudget <= userCost)
+                .isPresent();
+    }
+
+    protected String getUserOrDefault(String user) {
+        return StringUtils.isNotBlank(user) ? user : SHARED_RESOURCE_NAME;
+    }
+
+    private Integer toPercentage(Supplier<Optional<Integer>> allowedBudget, Double totalCost) {
+        return allowedBudget.get()
+                .map(userBudget -> (totalCost * ONE_HUNDRED) / userBudget)
+                .map(Double::intValue)
+                .orElse(BigDecimal.ZERO.intValue());
+    }
+
+    private List<Bson> matchCriteria(BillingFilter filter) {
+
+        List<Bson> searchCriteria = new ArrayList<>();
+
+        if (filter.getUser() != null && !filter.getUser().isEmpty()) {
+            searchCriteria.add(Filters.in(MongoKeyWords.DLAB_USER, filter.getUser()));
+        }
+
+        if (filter.getResourceType() != null && !filter.getResourceType().isEmpty()) {
+            searchCriteria.add(Filters.in("dlab_resource_type",
+                    DlabResourceType.getResourceTypeIds(filter.getResourceType())));
+        }
+
+        if (filter.getDlabId() != null && !filter.getDlabId().isEmpty()) {
+            searchCriteria.add(regex(dlabIdFieldName(), filter.getDlabId(), "i"));
+        }
+
+        if (filter.getDateStart() != null && !filter.getDateStart().isEmpty()) {
+            searchCriteria.add(gte(MongoKeyWords.USAGE_DAY, filter.getDateStart()));
+            searchCriteria.add(gte(FIELD_USAGE_DATE, filter.getDateStart()));
+        }
+        if (filter.getDateEnd() != null && !filter.getDateEnd().isEmpty()) {
+            searchCriteria.add(lte(MongoKeyWords.USAGE_DAY, filter.getDateEnd()));
+            searchCriteria.add(lte(FIELD_USAGE_DATE, filter.getDateEnd()));
+        }
+
+        searchCriteria.addAll(cloudMatchCriteria((T) filter));
+        return searchCriteria;
+    }
+
+    protected abstract List<Bson> cloudMatchCriteria(T filter);
+
+
+    private Optional<ShapeInfo> getComputationalShape(List<String> shapeNames, Document c) {
+        return isDataEngine(c.getString(DATAENGINE_DOCKER_IMAGE)) ? getDataEngineShape(shapeNames, c) :
+                getDataEngineServiceShape(shapeNames, c);
+    }
+
+    private Double aggregateBillingData(List<Bson> pipeline) {
+        return Optional.ofNullable(aggregate(BILLING, pipeline).first())
+                .map(d -> d.getDouble(TOTAL_FIELD_NAME))
+                .orElse(BigDecimal.ZERO.doubleValue());
+    }
+
+    private FindIterable<Document> getUserInstances() {
+        return getCollection(USER_INSTANCES)
+                .find()
+                .projection(
+                        fields(excludeId(),
+                                include(SHAPE, EXPLORATORY_ID, STATUS,
+                                        COMPUTATIONAL_RESOURCES + "." + COMPUTATIONAL_ID,
+                                        COMPUTATIONAL_RESOURCES + "." + MASTER_NODE_SHAPE,
+                                        COMPUTATIONAL_RESOURCES + "." + SLAVE_NODE_SHAPE,
+                                        COMPUTATIONAL_RESOURCES + "." + TOTAL_INSTANCE_NUMBER,
+                                        COMPUTATIONAL_RESOURCES + "." + DATAENGINE_SHAPE,
+                                        COMPUTATIONAL_RESOURCES + "." + DATAENGINE_INSTANCE_COUNT,
+                                        COMPUTATIONAL_RESOURCES + "." + DATAENGINE_DOCKER_IMAGE,
+                                        COMPUTATIONAL_RESOURCES + "." + STATUS
+                                )));
+    }
+
+    private Optional<ShapeInfo> getExploratoryShape(List<String> shapeNames, Document d) {
+        final String shape = d.getString(SHAPE);
+        if (isShapeAcceptable(shapeNames, shape)) {
+            return Optional.of(new ShapeInfo(shape, UserInstanceStatus.of(d.getString(STATUS))));
+        }
+        return Optional.empty();
+    }
+
+    private boolean isDataEngine(String dockerImage) {
+        return DataEngineType.fromDockerImageName(dockerImage) == DataEngineType.SPARK_STANDALONE;
+    }
+
+    private Optional<ShapeInfo> getDataEngineServiceShape(List<String> shapeNames,
+                                                          Document c) {
+        final String desMasterShape = c.getString(MASTER_NODE_SHAPE);
+        final String desSlaveShape = c.getString(SLAVE_NODE_SHAPE);
+        if (isShapeAcceptable(shapeNames, desMasterShape, desSlaveShape)) {
+            return Optional.of(new ShapeInfo(desMasterShape, desSlaveShape, c.getString(TOTAL_INSTANCE_NUMBER),
+                    UserInstanceStatus.of(c.getString(STATUS))));
+        }
+        return Optional.empty();
+    }
+
+    private Optional<ShapeInfo> getDataEngineShape(List<String> shapeNames, Document c) {
+        final String dataEngineShape = c.getString(DATAENGINE_SHAPE);
+        if ((isShapeAcceptable(shapeNames, dataEngineShape))
+                && StringUtils.isNotEmpty(c.getString(COMPUTATIONAL_ID))) {
+
+            return Optional.of(new ShapeInfo(dataEngineShape, c.getString(DATAENGINE_INSTANCE_COUNT),
+                    UserInstanceStatus.of(c.getString(STATUS))));
+        }
+        return Optional.empty();
+    }
+
+    private boolean isShapeAcceptable(List<String> shapeNames, String... shapes) {
+        return shapeNames == null || shapeNames.isEmpty() || Arrays.stream(shapes).anyMatch(shapeNames::contains);
+    }
+
+    protected void appendSsnAndEdgeNodeType(List<String> shapeNames, Map<String, ShapeInfo> shapes) {
+        final String ssnShape = getSsnShape();
+        if (shapeNames == null || shapeNames.isEmpty() || shapeNames.contains(ssnShape)) {
+            String serviceBaseName = getServiceBaseName();
+            shapes.put(serviceBaseName + "-ssn", new ShapeInfo(ssnShape, UserInstanceStatus.RUNNING));
+            FindIterable<Document> docs = getCollection(USER_EDGE)
+                    .find()
+                    .projection(fields(include(ID, EDGE_STATUS)));
+            for (Document d : docs) {
+                shapes.put(edgeId(d),
+                        new ShapeInfo(getEdgeSize(), UserInstanceStatus.of(d.getString(EDGE_STATUS))));
+            }
+        }
+    }
+
+    protected String getServiceBaseName() {
+        return settings.getServiceBaseName();
+    }
+
+    protected abstract String getEdgeSize();
+
+    protected abstract String edgeId(Document d);
+
+    protected abstract String getSsnShape();
+
+
+    protected String generateShapeName(ShapeInfo shape) {
+        return Optional.ofNullable(shape).map(ShapeInfo::getName).orElse(StringUtils.EMPTY);
+    }
+
+    protected void usersToLowerCase(List<String> users) {
+        if (users != null) {
+            users.replaceAll(u -> u != null ? u.toLowerCase() : null);
+        }
+    }
+
+    protected void setUserFilter(UserInfo userInfo, BillingFilter filter, boolean isFullReport) {
+        if (isFullReport) {
+            usersToLowerCase(filter.getUser());
+        } else {
+            filter.setUser(Lists.newArrayList(userInfo.getName().toLowerCase()));
+        }
+    }
+
+    /**
+     * Store shape info
+     */
+    @Getter
+    @ToString
+    protected class ShapeInfo {
+        private static final String DES_NAME_FORMAT = "Master: %s%sSlave:  %d x %s";
+        private static final String DE_NAME_FORMAT = "%d x %s";
+        private final boolean isDataEngine;
+        private final String shape;
+        private final String slaveShape;
+        private final String slaveCount;
+        private final boolean isExploratory;
+        private final UserInstanceStatus status;
+
+        private ShapeInfo(boolean isDataEngine, String shape, String slaveShape, String slaveCount, boolean
+                isExploratory, UserInstanceStatus status) {
+            this.isDataEngine = isDataEngine;
+            this.shape = shape;
+            this.slaveShape = slaveShape;
+            this.slaveCount = slaveCount;
+            this.isExploratory = isExploratory;
+            this.status = status;
+        }
+
+        public ShapeInfo(String shape, UserInstanceStatus status) {
+            this(false, shape, null, null, true, status);
+        }
+
+        ShapeInfo(String shape, String slaveShape, String slaveCount, UserInstanceStatus status) {
+            this(false, shape, slaveShape, slaveCount, false, status);
+        }
+
+
+        ShapeInfo(String shape, String slaveCount, UserInstanceStatus status) {
+            this(true, shape, null, slaveCount, false, status);
+        }
+
+        public String getName() {
+            if (isExploratory) {
+                return shape;
+            } else {
+                return clusterName();
+            }
+        }
+
+        private String clusterName() {
+            try {
+                final Integer count = Integer.valueOf(slaveCount);
+                return isDataEngine ? String.format(DE_NAME_FORMAT, count, shape) :
+                        String.format(DES_NAME_FORMAT, shape, System.lineSeparator(), count - 1, slaveShape);
+            } catch (NumberFormatException e) {
+                log.error("Cannot parse string {} to integer", slaveCount);
+                return StringUtils.EMPTY;
+            }
+        }
+    }
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/aws/AwsBillingDAO.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/aws/AwsBillingDAO.java
index 0e186c0..fa898b5 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/aws/AwsBillingDAO.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/aws/AwsBillingDAO.java
@@ -16,188 +16,67 @@
 
 package com.epam.dlab.backendapi.dao.aws;
 
-import com.epam.dlab.auth.UserInfo;
 import com.epam.dlab.backendapi.dao.BaseBillingDAO;
 import com.epam.dlab.backendapi.resources.dto.aws.AwsBillingFilter;
-import com.epam.dlab.backendapi.roles.RoleType;
-import com.epam.dlab.backendapi.roles.UserRoles;
-import com.epam.dlab.billing.BillingCalculationUtils;
-import com.epam.dlab.billing.DlabResourceType;
-import com.epam.dlab.dto.UserInstanceStatus;
 import com.epam.dlab.util.UsernameUtils;
-import com.mongodb.client.AggregateIterable;
-import com.mongodb.client.FindIterable;
-import org.apache.commons.lang3.StringUtils;
 import org.bson.Document;
 import org.bson.conversions.Bson;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
-import java.util.Map;
-import java.util.Optional;
 
-import static com.epam.dlab.backendapi.dao.MongoCollections.BILLING;
-import static com.epam.dlab.backendapi.dao.MongoCollections.USER_EDGE;
 import static com.epam.dlab.model.aws.ReportLine.*;
 import static com.mongodb.client.model.Accumulators.*;
-import static com.mongodb.client.model.Aggregates.*;
-import static com.mongodb.client.model.Filters.*;
-import static com.mongodb.client.model.Projections.fields;
-import static com.mongodb.client.model.Projections.include;
+import static com.mongodb.client.model.Aggregates.group;
+import static com.mongodb.client.model.Aggregates.sort;
 
 /**
  * DAO for user billing.
  */
 public class AwsBillingDAO extends BaseBillingDAO<AwsBillingFilter> {
-    private static final Logger LOGGER = LoggerFactory.getLogger(AwsBillingDAO.class);
 
     public static final String DLAB_RESOURCE_TYPE = "dlab_resource_type";
     public static final String USAGE_DATE_START = "usage_date_start";
     public static final String USAGE_DATE_END = "usage_date_end";
     public static final String TAG_RESOURCE_ID = "tag_resource_id";
 
-    /**
-     * Add the conditions to the list.
-     *
-     * @param conditions the list of conditions.
-     * @param fieldName  the name of field.
-     * @param values     the values.
-     */
-    private void addCondition(List<Bson> conditions, String fieldName, List<String> values) {
-        if (values != null && !values.isEmpty()) {
-            conditions.add(in(fieldName, values));
-        }
+    @Override
+    protected Bson sortCriteria() {
+        return sort(new Document(ID + "." + USER, 1)
+                .append(ID + "." + FIELD_DLAB_ID, 1)
+                .append(ID + "." + DLAB_RESOURCE_TYPE, 1)
+                .append(ID + "." + FIELD_PRODUCT, 1));
     }
 
-    /**
-     * Build and returns the billing report.
-     *
-     * @param userInfo user info
-     * @param filter   the filter for report data.
-     * @return billing report
-     */
-    public Document getReport(UserInfo userInfo, AwsBillingFilter filter) {
-        // Create filter
-        List<Bson> conditions = new ArrayList<>();
-        boolean isFullReport = UserRoles.checkAccess(userInfo, RoleType.PAGE, "/api/infrastructure_provision/billing");
-        setUserFilter(userInfo, filter, isFullReport);
-        addCondition(conditions, USER, filter.getUser());
-        addCondition(conditions, FIELD_PRODUCT, filter.getProduct());
-        addCondition(conditions, DLAB_RESOURCE_TYPE, DlabResourceType.getResourceTypeIds(filter.getResourceType()));
-
-        addAnotherConditionsIfNecessary(conditions, filter);
-
-        // Create aggregation conditions
-
-        List<Bson> pipeline = new ArrayList<>();
-        if (!conditions.isEmpty()) {
-            LOGGER.trace("Filter conditions is {}", conditions);
-            pipeline.add(match(and(conditions)));
-        }
-        pipeline.add(
-                group(getGroupingFields(USER, FIELD_DLAB_ID, DLAB_RESOURCE_TYPE, FIELD_PRODUCT, FIELD_RESOURCE_TYPE,
-                        FIELD_CURRENCY_CODE),
-                        sum(FIELD_COST, "$" + FIELD_COST),
-                        min(USAGE_DATE_START, "$" + FIELD_USAGE_DATE),
-                        max(USAGE_DATE_END, "$" + FIELD_USAGE_DATE)
-                ));
-        pipeline.add(
-                sort(new Document(ID + "." + USER, 1)
-                        .append(ID + "." + FIELD_DLAB_ID, 1)
-                        .append(ID + "." + DLAB_RESOURCE_TYPE, 1)
-                        .append(ID + "." + FIELD_PRODUCT, 1))
-        );
-
-        // Get billing report and the list of shape info
-        AggregateIterable<Document> agg = getCollection(BILLING).aggregate(pipeline);
-        Map<String, ShapeInfo> shapes = getShapes(filter.getShape());
-
-        // Build billing report lines
-        List<Document> reportItems = new ArrayList<>();
-        boolean filterByShape = !(filter.getShape() == null || filter.getShape().isEmpty());
-        String usageDateStart = null;
-        String usageDateEnd = null;
-        double costTotal = 0;
-
-        for (Document d : agg) {
-            Document id = (Document) d.get(ID);
-            String resourceId = id.getString(FIELD_DLAB_ID);
-            ShapeInfo shape = shapes.get(resourceId);
-            final UserInstanceStatus status = Optional.ofNullable(shape).map(ShapeInfo::getStatus).orElse(null);
-            if ((filterByShape && shape == null) || (!filter.getStatuses().isEmpty() && filter.getStatuses().stream()
-                    .noneMatch(s -> s.equals(status)))) {
-                continue;
-            }
-
-            String resourceTypeId = DlabResourceType.getResourceTypeName(id.getString(DLAB_RESOURCE_TYPE));
-            String shapeName = generateShapeName(shape);
-            String dateStart = d.getString(USAGE_DATE_START);
-            if (StringUtils.compare(usageDateStart, dateStart, false) > 0) {
-                usageDateStart = dateStart;
-            }
-            String dateEnd = d.getString(USAGE_DATE_END);
-            if (StringUtils.compare(usageDateEnd, dateEnd) < 0) {
-                usageDateEnd = dateEnd;
-            }
-            double cost = BillingCalculationUtils.round(d.getDouble(FIELD_COST), 2);
-            costTotal += cost;
-
-            Document item = new Document()
-                    .append(FIELD_USER_ID, getUserOrDefault(id.getString(USER)))
-                    .append(FIELD_DLAB_ID, resourceId)
-                    .append(DLAB_RESOURCE_TYPE, resourceTypeId)
-                    .append(SHAPE, shapeName)
-                    .append(STATUS,
-                            Optional.ofNullable(status).map(UserInstanceStatus::toString).orElse(StringUtils.EMPTY))
-                    .append(FIELD_PRODUCT, id.getString(FIELD_PRODUCT))
-                    .append(FIELD_RESOURCE_TYPE, id.getString(FIELD_RESOURCE_TYPE))
-                    .append(FIELD_COST, BillingCalculationUtils.formatDouble(cost))
-                    .append(FIELD_CURRENCY_CODE, id.getString(FIELD_CURRENCY_CODE))
-                    .append(USAGE_DATE_START, dateStart)
-                    .append(USAGE_DATE_END, dateEnd);
-            reportItems.add(item);
-        }
-
-        return new Document()
-                .append(SERVICE_BASE_NAME, settings.getServiceBaseName())
-                .append(TAG_RESOURCE_ID, settings.getConfTagResourceId())
-                .append(USAGE_DATE_START, usageDateStart)
-                .append(USAGE_DATE_END, usageDateEnd)
-                .append(ITEMS, reportItems)
-                .append(COST_TOTAL, BillingCalculationUtils.formatDouble(BillingCalculationUtils.round(costTotal, 2)))
-                .append(FIELD_CURRENCY_CODE, (reportItems.isEmpty() ? null :
-                        reportItems.get(0).getString(FIELD_CURRENCY_CODE)))
-                .append(FULL_REPORT, isFullReport);
+    @Override
+    protected Bson groupCriteria() {
+        return group(getGroupingFields(USER, FIELD_DLAB_ID, DLAB_RESOURCE_TYPE, FIELD_PRODUCT, FIELD_RESOURCE_TYPE,
+                FIELD_CURRENCY_CODE),
+                sum(FIELD_COST, "$" + FIELD_COST),
+                min(USAGE_DATE_START, "$" + FIELD_USAGE_DATE),
+                max(USAGE_DATE_END, "$" + FIELD_USAGE_DATE));
     }
 
-    private void addAnotherConditionsIfNecessary(List<Bson> conditions, AwsBillingFilter filter) {
-        if (filter.getDlabId() != null && !filter.getDlabId().isEmpty()) {
-            conditions.add(regex(FIELD_DLAB_ID, filter.getDlabId(), "i"));
-        }
-
-        if (filter.getDateStart() != null && !filter.getDateStart().isEmpty()) {
-            conditions.add(gte(FIELD_USAGE_DATE, filter.getDateStart()));
-        }
-        if (filter.getDateEnd() != null && !filter.getDateEnd().isEmpty()) {
-            conditions.add(lte(FIELD_USAGE_DATE, filter.getDateEnd()));
-        }
+    @Override
+    protected List<Bson> cloudMatchCriteria(AwsBillingFilter filter) {
+        return Collections.emptyList();
     }
 
-    protected void appendSsnAndEdgeNodeType(List<String> shapeNames, Map<String, ShapeInfo> shapes) {
-        // Add SSN and EDGE nodes
-        final String ssnShape = "t2.medium";
-        if (shapeNames == null || shapeNames.isEmpty() || shapeNames.contains(ssnShape)) {
-            String serviceBaseName = settings.getServiceBaseName();
-            shapes.put(serviceBaseName + "-ssn", new ShapeInfo(ssnShape, UserInstanceStatus.RUNNING));
-            FindIterable<Document> docs = getCollection(USER_EDGE)
-                    .find()
-                    .projection(fields(include(ID, EDGE_STATUS)));
-            for (Document d : docs) {
-                shapes.put(String.join("-", serviceBaseName, UsernameUtils.removeDomain(d.getString(ID)), "edge"),
-                        new ShapeInfo(ssnShape, UserInstanceStatus.of(d.getString(EDGE_STATUS))));
-            }
-        }
+    @Override
+    protected String getEdgeSize() {
+        return getSsnShape();
     }
+
+    public String edgeId(Document d) {
+        return String.join("-", settings.getServiceBaseName(), UsernameUtils.removeDomain(d.getString(ID)), "edge");
+    }
+
+
+    @Override
+    protected String getSsnShape() {
+        return "t2.medium";
+    }
+
+
+
 }
\ No newline at end of file
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/azure/AzureBillingDAO.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/azure/AzureBillingDAO.java
index 401c83c..c241b06 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/azure/AzureBillingDAO.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/azure/AzureBillingDAO.java
@@ -17,156 +17,36 @@
 package com.epam.dlab.backendapi.dao.azure;
 
 import com.epam.dlab.MongoKeyWords;
-import com.epam.dlab.auth.UserInfo;
 import com.epam.dlab.backendapi.dao.BaseBillingDAO;
 import com.epam.dlab.backendapi.resources.dto.azure.AzureBillingFilter;
-import com.epam.dlab.backendapi.roles.RoleType;
-import com.epam.dlab.backendapi.roles.UserRoles;
-import com.epam.dlab.billing.BillingCalculationUtils;
-import com.epam.dlab.billing.DlabResourceType;
-import com.epam.dlab.dto.UserInstanceStatus;
 import com.google.inject.Singleton;
-import com.mongodb.client.AggregateIterable;
-import com.mongodb.client.FindIterable;
 import com.mongodb.client.model.Accumulators;
 import com.mongodb.client.model.Aggregates;
 import com.mongodb.client.model.Filters;
 import com.mongodb.client.model.Sorts;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.commons.lang3.StringUtils;
 import org.bson.Document;
 import org.bson.conversions.Bson;
 
-import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import static com.epam.dlab.backendapi.dao.MongoCollections.USER_EDGE;
-import static com.mongodb.client.model.Filters.*;
-import static com.mongodb.client.model.Projections.fields;
-import static com.mongodb.client.model.Projections.include;
 
 @Singleton
 @Slf4j
 public class AzureBillingDAO extends BaseBillingDAO<AzureBillingFilter> {
     public static final String SIZE = "size";
 
-    public Document  getReport(UserInfo userInfo, AzureBillingFilter filter) {
-
-        boolean isFullReport = UserRoles.checkAccess(userInfo, RoleType.PAGE, "/api/infrastructure_provision/billing");
-        setUserFilter(userInfo, filter, isFullReport);
-
-        List<Bson> matchCriteria = matchCriteria(filter);
-        List<Bson> pipeline = new ArrayList<>();
-        if (!matchCriteria.isEmpty()) {
-            pipeline.add(Aggregates.match(Filters.and(matchCriteria)));
-        }
-        pipeline.add(groupCriteria());
-        pipeline.add(sortCriteria());
-
-        return prepareReport(
-                filter.getStatuses(), filter.getNodeSize() != null && !filter.getNodeSize().isEmpty(),
-                getCollection(MongoKeyWords.BILLING_DETAILS).aggregate(pipeline),
-                getShapes(filter.getNodeSize()))
-                .append(FULL_REPORT, isFullReport);
-    }
-
-    private Document prepareReport(List<UserInstanceStatus> statuses, boolean filterByShape,
-                                   AggregateIterable<Document> agg,
-                                   Map<String, ShapeInfo> shapes) {
-
-        List<Document> reportItems = new ArrayList<>();
-
-        String usageDateStart = null;
-        String usageDateEnd = null;
-        double costTotal = 0D;
-
-        for (Document d : agg) {
-            Document id = (Document) d.get(MongoKeyWords.MONGO_ID);
-            String resourceId = id.getString(MongoKeyWords.DLAB_ID);
-            ShapeInfo shape = shapes.get(resourceId);
-            final UserInstanceStatus status = Optional.ofNullable(shape).map(ShapeInfo::getStatus).orElse(null);
-            if ((filterByShape && shape == null) ||
-                    (!statuses.isEmpty() && statuses.stream().noneMatch(s -> s.equals(status)))) {
-                continue;
-            }
-
-            String dateStart = d.getString(MongoKeyWords.USAGE_FROM);
-            if (StringUtils.compare(usageDateStart, dateStart, false) > 0) {
-                usageDateStart = dateStart;
-            }
-            String dateEnd = d.getString(MongoKeyWords.USAGE_TO);
-            if (StringUtils.compare(usageDateEnd, dateEnd) < 0) {
-                usageDateEnd = dateEnd;
-            }
-
-            costTotal += d.getDouble(MongoKeyWords.COST);
-
-            Document item = new Document()
-                    .append(MongoKeyWords.DLAB_USER, getUserOrDefault(id.getString(USER)))
-                    .append(MongoKeyWords.DLAB_ID, resourceId)
-                    .append(SIZE, generateShapeName(shape))
-                    .append(STATUS,
-                            Optional.ofNullable(status).map(UserInstanceStatus::toString).orElse(StringUtils.EMPTY))
-                    .append(MongoKeyWords.METER_CATEGORY, id.getString(MongoKeyWords.METER_CATEGORY))
-                    .append(MongoKeyWords.RESOURCE_TYPE,
-                            DlabResourceType.getResourceTypeName(id.getString(MongoKeyWords.RESOURCE_TYPE)))
-                    .append(MongoKeyWords.COST, d.getDouble(MongoKeyWords.COST))
-                    .append(MongoKeyWords.COST_STRING, BillingCalculationUtils.formatDouble(d.getDouble(MongoKeyWords
-                            .COST)))
-                    .append(MongoKeyWords.CURRENCY_CODE, id.getString(MongoKeyWords.CURRENCY_CODE))
-                    .append(MongoKeyWords.USAGE_FROM, dateStart)
-                    .append(MongoKeyWords.USAGE_TO, dateEnd);
-
-
-            reportItems.add(item);
-        }
-
-        return new Document()
-                .append(SERVICE_BASE_NAME, settings.getServiceBaseName())
-                .append(MongoKeyWords.USAGE_FROM, usageDateStart)
-                .append(MongoKeyWords.USAGE_TO, usageDateEnd)
-                .append(ITEMS, reportItems)
-                .append(MongoKeyWords.COST_STRING, BillingCalculationUtils.formatDouble(BillingCalculationUtils.round
-                        (costTotal, 2)))
-                .append(MongoKeyWords.CURRENCY_CODE, (reportItems.isEmpty() ? null :
-                        reportItems.get(0).getString(MongoKeyWords.CURRENCY_CODE)));
-
-    }
-
-    private List<Bson> matchCriteria(AzureBillingFilter filter) {
-
-        List<Bson> searchCriteria = new ArrayList<>();
-
-        if (filter.getUser() != null && !filter.getUser().isEmpty()) {
-            searchCriteria.add(Filters.in(MongoKeyWords.DLAB_USER, filter.getUser()));
-        }
-
-        if (filter.getCategory() != null && !filter.getCategory().isEmpty()) {
-            searchCriteria.add(Filters.in(MongoKeyWords.METER_CATEGORY, filter.getCategory()));
-        }
-
-        if (filter.getResourceType() != null && !filter.getResourceType().isEmpty()) {
-            searchCriteria.add(Filters.in(MongoKeyWords.RESOURCE_TYPE,
-                    DlabResourceType.getResourceTypeIds(filter.getResourceType())));
-        }
-
-        if (filter.getDlabId() != null && !filter.getDlabId().isEmpty()) {
-            searchCriteria.add(regex(MongoKeyWords.DLAB_ID, filter.getDlabId(), "i"));
-        }
-
-        if (filter.getDateStart() != null && !filter.getDateStart().isEmpty()) {
-            searchCriteria.add(gte(MongoKeyWords.USAGE_DAY, filter.getDateStart()));
-        }
-        if (filter.getDateEnd() != null && !filter.getDateEnd().isEmpty()) {
-            searchCriteria.add(lte(MongoKeyWords.USAGE_DAY, filter.getDateEnd()));
+    @Override
+    protected List<Bson> cloudMatchCriteria(AzureBillingFilter filter) {
+        if (!filter.getCategory().isEmpty()) {
+            return Collections.singletonList(Filters.in(MongoKeyWords.METER_CATEGORY, filter.getCategory()));
+        } else {
+            return Collections.emptyList();
         }
-
-        return searchCriteria;
     }
 
-    private Bson groupCriteria() {
+    @Override
+    protected Bson groupCriteria() {
         return Aggregates.group(getGroupingFields(
                 MongoKeyWords.DLAB_USER,
                 MongoKeyWords.DLAB_ID,
@@ -179,7 +59,8 @@ public class AzureBillingDAO extends BaseBillingDAO<AzureBillingFilter> {
         );
     }
 
-    private Bson sortCriteria() {
+    @Override
+    protected Bson sortCriteria() {
         return Aggregates.sort(Sorts.ascending(
                 MongoKeyWords.prependId(MongoKeyWords.DLAB_USER),
                 MongoKeyWords.prependId(MongoKeyWords.DLAB_ID),
@@ -188,25 +69,57 @@ public class AzureBillingDAO extends BaseBillingDAO<AzureBillingFilter> {
     }
 
     @Override
-    protected void appendSsnAndEdgeNodeType(List<String> shapeNames, Map<String, ShapeInfo> shapes) {
+    protected String getServiceBaseName() {
+        return settings.getServiceBaseName().replace("_", "-").toLowerCase();
+    }
+
+    @Override
+    protected String getEdgeSize() {
+        return settings.getAzureEdgeInstanceSize();
+    }
 
-        String serviceBaseName = settings.getServiceBaseName().replace("_", "-").toLowerCase();
+    @Override
+    protected String edgeId(Document d) {
+        return d.getString(INSTANCE_ID);
+    }
 
-        final String ssnSize = settings.getAzureSsnInstanceSize();
-        if (shapeNames == null || shapeNames.isEmpty() || shapeNames.contains(ssnSize)) {
-            shapes.put(serviceBaseName + "-ssn", new BaseBillingDAO.ShapeInfo(ssnSize, UserInstanceStatus.RUNNING));
-        }
+    @Override
+    protected String getSsnShape() {
+        return settings.getAzureSsnInstanceSize();
+    }
 
+    @Override
+    protected String shapeFieldName() {
+        return SIZE;
+    }
 
-        final String edgeSize = settings.getAzureEdgeInstanceSize();
-        if (shapeNames == null || shapeNames.isEmpty() || shapeNames.contains(edgeSize)) {
-            FindIterable<Document> docs = getCollection(USER_EDGE)
-                    .find()
-                    .projection(fields(include(INSTANCE_ID, EDGE_STATUS)));
-            for (Document d : docs) {
-                shapes.put(d.getString(INSTANCE_ID),
-                        new BaseBillingDAO.ShapeInfo(edgeSize, UserInstanceStatus.of(d.getString(EDGE_STATUS))));
-            }
-        }
+    @Override
+    protected String dlabIdFieldName() {
+        return MongoKeyWords.DLAB_ID;
+    }
+
+    @Override
+    protected String productFieldName() {
+        return MongoKeyWords.METER_CATEGORY;
+    }
+
+    @Override
+    protected String costFieldName() {
+        return MongoKeyWords.COST_STRING;
+    }
+
+    @Override
+    protected String usageDateFromFieldName() {
+        return MongoKeyWords.USAGE_FROM;
+    }
+
+    @Override
+    protected String usageDateToFieldName() {
+        return MongoKeyWords.USAGE_TO;
+    }
+
+    @Override
+    protected String currencyCodeFieldName() {
+        return MongoKeyWords.CURRENCY_CODE;
     }
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/gcp/GcpBillingDao.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/gcp/GcpBillingDao.java
index 4a51272..ed7f6f6 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/gcp/GcpBillingDao.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/gcp/GcpBillingDao.java
@@ -18,36 +18,42 @@
 
 package com.epam.dlab.backendapi.dao.gcp;
 
-import com.epam.dlab.auth.UserInfo;
 import com.epam.dlab.backendapi.dao.BaseBillingDAO;
 import com.epam.dlab.backendapi.resources.dto.gcp.GcpBillingFilter;
-import com.epam.dlab.backendapi.roles.RoleType;
-import com.epam.dlab.backendapi.roles.UserRoles;
-import com.epam.dlab.billing.BillingCalculationUtils;
-import com.epam.dlab.billing.DlabResourceType;
-import com.epam.dlab.dto.UserInstanceStatus;
-import com.mongodb.client.AggregateIterable;
-import org.apache.commons.lang3.StringUtils;
+import com.epam.dlab.util.UsernameUtils;
 import org.bson.Document;
 import org.bson.conversions.Bson;
 
-import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
-import java.util.Map;
-import java.util.Optional;
 
-import static com.epam.dlab.MongoKeyWords.*;
-import static com.epam.dlab.backendapi.dao.MongoCollections.BILLING;
+import static com.epam.dlab.MongoKeyWords.USAGE_FROM;
+import static com.epam.dlab.MongoKeyWords.USAGE_TO;
 import static com.epam.dlab.backendapi.dao.aws.AwsBillingDAO.DLAB_RESOURCE_TYPE;
-import static com.epam.dlab.backendapi.dao.aws.AwsBillingDAO.TAG_RESOURCE_ID;
 import static com.epam.dlab.model.aws.ReportLine.*;
 import static com.mongodb.client.model.Accumulators.*;
-import static com.mongodb.client.model.Aggregates.*;
-import static com.mongodb.client.model.Filters.and;
-import static com.mongodb.client.model.Filters.in;
+import static com.mongodb.client.model.Aggregates.group;
+import static com.mongodb.client.model.Aggregates.sort;
 
 public class GcpBillingDao extends BaseBillingDAO<GcpBillingFilter> {
     @Override
+    protected Bson sortCriteria() {
+        return sort(new Document(ID + "." + USER, 1)
+                .append(ID + "." + FIELD_DLAB_ID, 1)
+                .append(ID + "." + FIELD_PRODUCT, 1));
+    }
+
+    @Override
+    protected Bson groupCriteria() {
+        return group(getGroupingFields(USER, FIELD_DLAB_ID, DLAB_RESOURCE_TYPE, FIELD_PRODUCT,
+                currencyCodeFieldName()),
+                sum(FIELD_COST, "$" + FIELD_COST),
+                min(USAGE_FROM, "$" + FIELD_USAGE_DATE),
+                max(USAGE_TO, "$" + FIELD_USAGE_DATE)
+        );
+    }
+
+    @Override
     public Double getTotalCost() {
         return null;
     }
@@ -78,104 +84,22 @@ public class GcpBillingDao extends BaseBillingDAO<GcpBillingFilter> {
     }
 
     @Override
-    protected void appendSsnAndEdgeNodeType(List<String> shapeNames, Map<String, ShapeInfo> shapes) {
-
+    protected List<Bson> cloudMatchCriteria(GcpBillingFilter filter) {
+        return Collections.emptyList();
     }
 
-    public Document getReport(UserInfo userInfo, GcpBillingFilter filter) {
-        // Create filter
-        List<Bson> conditions = new ArrayList<>();
-        boolean isFullReport = UserRoles.checkAccess(userInfo, RoleType.PAGE, "/api/infrastructure_provision/billing");
-        setUserFilter(userInfo, filter, isFullReport);
-        addCondition(conditions, USER, filter.getUser());
-        addCondition(conditions, FIELD_PRODUCT, filter.getProduct());
-
-        // Create aggregation conditions
-
-        List<Bson> pipeline = new ArrayList<>();
-        if (!conditions.isEmpty()) {
-            pipeline.add(match(and(conditions)));
-        }
-        pipeline.add(
-                group(getGroupingFields(USER, FIELD_DLAB_ID, DLAB_RESOURCE_TYPE, FIELD_PRODUCT, FIELD_RESOURCE_TYPE,
-                        FIELD_CURRENCY_CODE),
-                        sum(FIELD_COST, "$" + FIELD_COST),
-                        min(USAGE_FROM, "$" + FIELD_USAGE_DATE),
-                        max(USAGE_TO, "$" + FIELD_USAGE_DATE)
-                ));
-        pipeline.add(
-                sort(new Document(ID + "." + USER, 1)
-                        .append(ID + "." + FIELD_DLAB_ID, 1)
-                        .append(ID + "." + RESOURCE_TYPE, 1)
-                        .append(ID + "." + FIELD_PRODUCT, 1))
-        );
-
-        // Get billing report and the list of shape info
-        AggregateIterable<Document> agg = getCollection(BILLING).aggregate(pipeline);
-        Map<String, ShapeInfo> shapes = getShapes(filter.getShape());
-
-        // Build billing report lines
-        List<Document> reportItems = new ArrayList<>();
-        boolean filterByShape = !(filter.getShape() == null || filter.getShape().isEmpty());
-        String usageDateStart = null;
-        String usageDateEnd = null;
-        double costTotal = 0;
-
-        for (Document d : agg) {
-            Document id = (Document) d.get(ID);
-            String resourceId = id.getString(FIELD_DLAB_ID);
-            ShapeInfo shape = shapes.get(resourceId);
-            final UserInstanceStatus status = Optional.ofNullable(shape).map(ShapeInfo::getStatus).orElse(null);
-            if ((filterByShape && shape == null) || (!filter.getStatuses().isEmpty() && filter.getStatuses().stream()
-                    .noneMatch(s -> s.equals(status)))) {
-                continue;
-            }
-
-            String resourceTypeId = DlabResourceType.getResourceTypeName(id.getString(DLAB_RESOURCE_TYPE));
-            String shapeName = generateShapeName(shape);
-            String dateStart = d.getString(USAGE_FROM);
-            if (StringUtils.compare(usageDateStart, dateStart, false) > 0) {
-                usageDateStart = dateStart;
-            }
-            String dateEnd = d.getString(USAGE_TO);
-            if (StringUtils.compare(usageDateEnd, dateEnd) < 0) {
-                usageDateEnd = dateEnd;
-            }
-            double cost = BillingCalculationUtils.round(d.getDouble(FIELD_COST), 2);
-            costTotal += cost;
-
-            Document item = new Document()
-                    .append(FIELD_USER_ID, getUserOrDefault(id.getString(USER)))
-                    .append(FIELD_DLAB_ID, resourceId)
-                    .append(DLAB_RESOURCE_TYPE, resourceTypeId)
-                    .append(SHAPE, shapeName)
-                    .append(STATUS,
-                            Optional.ofNullable(status).map(UserInstanceStatus::toString).orElse(StringUtils.EMPTY))
-                    .append(FIELD_PRODUCT, id.getString(FIELD_PRODUCT))
-                    .append(FIELD_RESOURCE_TYPE, id.getString(FIELD_RESOURCE_TYPE))
-                    .append(FIELD_COST, BillingCalculationUtils.formatDouble(cost))
-                    .append(FIELD_CURRENCY_CODE, id.getString(FIELD_CURRENCY_CODE))
-                    .append(USAGE_FROM, dateStart)
-                    .append(USAGE_TO, dateEnd);
-            reportItems.add(item);
-        }
-
-        return new Document()
-                .append(SERVICE_BASE_NAME, settings.getServiceBaseName())
-                .append(TAG_RESOURCE_ID, settings.getConfTagResourceId())
-                .append(USAGE_FROM, usageDateStart)
-                .append(USAGE_TO, usageDateEnd)
-                .append(ITEMS, reportItems)
-                .append(COST_TOTAL, BillingCalculationUtils.formatDouble(BillingCalculationUtils.round(costTotal, 2)))
-                .append(FIELD_CURRENCY_CODE, (reportItems.isEmpty() ? null :
-                        reportItems.get(0).getString(FIELD_CURRENCY_CODE)))
-                .append(FULL_REPORT, isFullReport);
+    @Override
+    protected String getEdgeSize() {
+        return getSsnShape();
     }
 
-    private void addCondition(List<Bson> conditions, String fieldName, List<String> values) {
-        if (values != null && !values.isEmpty()) {
-            conditions.add(in(fieldName, values));
-        }
+    public String edgeId(Document d) {
+        return String.join("-", settings.getServiceBaseName(), UsernameUtils.removeDomain(d.getString(ID)), "edge");
     }
 
+
+    @Override
+    protected String getSsnShape() {
+        return "t2.medium";
+    }
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/BillingFilter.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/BillingFilter.java
index 8c65c5a..d26722b 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/BillingFilter.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/BillingFilter.java
@@ -8,7 +8,7 @@ import java.util.Collections;
 import java.util.List;
 
 @Data
-public class BillingFilter {
+public abstract class BillingFilter {
 	@JsonProperty
 	protected List<String> user;
 	@JsonProperty("dlab_id")
@@ -21,4 +21,6 @@ public class BillingFilter {
 	protected String dateEnd;
 	@JsonProperty("status")
 	protected List<UserInstanceStatus> statuses = Collections.emptyList();
+
+	public abstract List<String> getShapes();
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/aws/AwsBillingFilter.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/aws/AwsBillingFilter.java
index 1371754..00b2017 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/aws/AwsBillingFilter.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/aws/AwsBillingFilter.java
@@ -31,4 +31,10 @@ public class AwsBillingFilter extends BillingFilter {
 	private List<String> product;
 	@JsonProperty
 	private List<String> shape;
+
+
+	@Override
+	public List<String> getShapes() {
+		return shape;
+	}
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/azure/AzureBillingFilter.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/azure/AzureBillingFilter.java
index d724459..0de5818 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/azure/AzureBillingFilter.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/azure/AzureBillingFilter.java
@@ -21,12 +21,18 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import lombok.Data;
 
+import java.util.Collections;
 import java.util.List;
 
 @Data
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class AzureBillingFilter extends BillingFilter {
-	@JsonProperty("size")
-	private List<String> nodeSize;
-	private List<String> category;
+    @JsonProperty("size")
+    private List<String> nodeSize;
+    private List<String> category = Collections.emptyList();
+
+    @Override
+    public List<String> getShapes() {
+        return nodeSize;
+    }
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/gcp/GcpBillingFilter.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/gcp/GcpBillingFilter.java
index 2966146..3d855f2 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/gcp/GcpBillingFilter.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/gcp/GcpBillingFilter.java
@@ -34,4 +34,9 @@ public class GcpBillingFilter extends BillingFilter {
     private List<String> shape;
     @JsonProperty
     private List<String> product;
+
+    @Override
+    public List<String> getShapes() {
+        return shape;
+    }
 }
diff --git a/services/self-service/src/main/resources/webapp/src/dictionary/azure.dictionary.ts b/services/self-service/src/main/resources/webapp/src/dictionary/azure.dictionary.ts
index f040e97..00379fc 100644
--- a/services/self-service/src/main/resources/webapp/src/dictionary/azure.dictionary.ts
+++ b/services/self-service/src/main/resources/webapp/src/dictionary/azure.dictionary.ts
@@ -36,7 +36,7 @@ export const NAMING_CONVENTION = {
     'billing': {
         'resourceName': 'resourceName',
         'cost': 'costString',
-        'costTotal': 'costString',
+        'costTotal': 'cost_total',
         'currencyCode': 'currencyCode',
         'dateFrom': 'from',
         'dateTo': 'to',
@@ -107,4 +107,4 @@ export class ReportingConfigModel {
         this.date_end = '';
         this.dlab_id = '';
     }
-}
\ No newline at end of file
+}
diff --git a/services/self-service/src/main/resources/webapp/src/dictionary/gcp.dictionary.ts b/services/self-service/src/main/resources/webapp/src/dictionary/gcp.dictionary.ts
index 7c7f10d..1bf1111 100644
--- a/services/self-service/src/main/resources/webapp/src/dictionary/gcp.dictionary.ts
+++ b/services/self-service/src/main/resources/webapp/src/dictionary/gcp.dictionary.ts
@@ -39,12 +39,12 @@ export const NAMING_CONVENTION = {
       'cost': 'cost',
       'costTotal': 'cost_total',
       'currencyCode': 'currency_code',
-      'dateFrom': 'usage_date_start',
-      'dateTo': 'usage_date_end',
+      'dateFrom': 'from',
+      'dateTo': 'to',
       'service': 'product',
       'service_filter_key': 'product',
-      'type': 'resource_type',
-      'resourceType': 'dlab_resource_type',
+      'type': 'dlab_resource_type',
+      'resourceType': 'resource_type',
       'instance_size': 'shape',
       'dlabId': 'dlab_id'
   },


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@dlab.apache.org
For additional commands, e-mail: commits-help@dlab.apache.org


[incubator-dlab] 01/02: DLAB-000 billing mock on gcp implemented

Posted by bh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bhliva pushed a commit to branch billing_gcp
in repository https://gitbox.apache.org/repos/asf/incubator-dlab.git

commit 310b9c55db6a6e9205934c274beb471b0dd2bad4
Author: bhliva <bo...@epam.com>
AuthorDate: Mon Mar 4 17:33:43 2019 +0200

    DLAB-000 billing mock on gcp implemented
---
 services/provisioning-service/provisioning.yml     |   4 +-
 .../dlab/backendapi/dao/azure/AzureBillingDAO.java |   2 +-
 .../dlab/backendapi/dao/gcp/GcpBillingDao.java     | 194 +++++++++++++++++----
 .../backendapi/modules/GcpSelfServiceModule.java   |  75 ++++----
 .../resources/dto/gcp/GcpBillingFilter.java        |  37 ++++
 .../resources/gcp/BillingResourceGcp.java          |  56 ++++++
 .../dlab/backendapi/service/BillingService.java    |  87 +++++----
 .../backendapi/service/aws/AwsBillingService.java  |  14 --
 .../service/azure/AzureBillingService.java         | 150 ++++++++--------
 .../backendapi/service/gcp/GcpBillingService.java  |  55 ++++++
 10 files changed, 468 insertions(+), 206 deletions(-)

diff --git a/services/provisioning-service/provisioning.yml b/services/provisioning-service/provisioning.yml
index e14ce94..a97dd4a 100644
--- a/services/provisioning-service/provisioning.yml
+++ b/services/provisioning-service/provisioning.yml
@@ -56,7 +56,7 @@ server:
     - type: https
       port: 8084
       certAlias: dlab
-      validateCerts: true
+      validateCerts: false
       keyStorePath: ${KEY_STORE_PATH}
       keyStorePassword: ${KEY_STORE_PASSWORD}
       trustStorePath: ${TRUST_STORE_PATH}
@@ -66,7 +66,7 @@ server:
     - type: https
       port: 8085
       certAlias: dlab
-      validateCerts: true
+      validateCerts: false
       keyStorePath: ${KEY_STORE_PATH}
       keyStorePassword: ${KEY_STORE_PASSWORD}
       trustStorePath: ${TRUST_STORE_PATH}
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/azure/AzureBillingDAO.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/azure/AzureBillingDAO.java
index ad818f8..401c83c 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/azure/AzureBillingDAO.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/azure/AzureBillingDAO.java
@@ -52,7 +52,7 @@ import static com.mongodb.client.model.Projections.include;
 public class AzureBillingDAO extends BaseBillingDAO<AzureBillingFilter> {
     public static final String SIZE = "size";
 
-    public Document getReport(UserInfo userInfo, AzureBillingFilter filter) {
+    public Document  getReport(UserInfo userInfo, AzureBillingFilter filter) {
 
         boolean isFullReport = UserRoles.checkAccess(userInfo, RoleType.PAGE, "/api/infrastructure_provision/billing");
         setUserFilter(userInfo, filter, isFullReport);
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/gcp/GcpBillingDao.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/gcp/GcpBillingDao.java
index 1bb4215..4a51272 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/gcp/GcpBillingDao.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/dao/gcp/GcpBillingDao.java
@@ -19,43 +19,163 @@
 package com.epam.dlab.backendapi.dao.gcp;
 
 import com.epam.dlab.auth.UserInfo;
-import com.epam.dlab.backendapi.dao.BillingDAO;
-import com.epam.dlab.backendapi.resources.dto.BillingFilter;
+import com.epam.dlab.backendapi.dao.BaseBillingDAO;
+import com.epam.dlab.backendapi.resources.dto.gcp.GcpBillingFilter;
+import com.epam.dlab.backendapi.roles.RoleType;
+import com.epam.dlab.backendapi.roles.UserRoles;
+import com.epam.dlab.billing.BillingCalculationUtils;
+import com.epam.dlab.billing.DlabResourceType;
+import com.epam.dlab.dto.UserInstanceStatus;
+import com.mongodb.client.AggregateIterable;
+import org.apache.commons.lang3.StringUtils;
 import org.bson.Document;
+import org.bson.conversions.Bson;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static com.epam.dlab.MongoKeyWords.*;
+import static com.epam.dlab.backendapi.dao.MongoCollections.BILLING;
+import static com.epam.dlab.backendapi.dao.aws.AwsBillingDAO.DLAB_RESOURCE_TYPE;
+import static com.epam.dlab.backendapi.dao.aws.AwsBillingDAO.TAG_RESOURCE_ID;
+import static com.epam.dlab.model.aws.ReportLine.*;
+import static com.mongodb.client.model.Accumulators.*;
+import static com.mongodb.client.model.Aggregates.*;
+import static com.mongodb.client.model.Filters.and;
+import static com.mongodb.client.model.Filters.in;
+
+public class GcpBillingDao extends BaseBillingDAO<GcpBillingFilter> {
+    @Override
+    public Double getTotalCost() {
+        return null;
+    }
+
+    @Override
+    public Double getUserCost(String user) {
+        return null;
+    }
+
+    @Override
+    public int getBillingQuoteUsed() {
+        return 0;
+    }
+
+    @Override
+    public int getBillingUserQuoteUsed(String user) {
+        return 0;
+    }
+
+    @Override
+    public boolean isBillingQuoteReached() {
+        return false;
+    }
+
+    @Override
+    public boolean isUserQuoteReached(String user) {
+        return false;
+    }
+
+    @Override
+    protected void appendSsnAndEdgeNodeType(List<String> shapeNames, Map<String, ShapeInfo> shapes) {
+
+    }
+
+    public Document getReport(UserInfo userInfo, GcpBillingFilter filter) {
+        // Create filter
+        List<Bson> conditions = new ArrayList<>();
+        boolean isFullReport = UserRoles.checkAccess(userInfo, RoleType.PAGE, "/api/infrastructure_provision/billing");
+        setUserFilter(userInfo, filter, isFullReport);
+        addCondition(conditions, USER, filter.getUser());
+        addCondition(conditions, FIELD_PRODUCT, filter.getProduct());
+
+        // Create aggregation conditions
+
+        List<Bson> pipeline = new ArrayList<>();
+        if (!conditions.isEmpty()) {
+            pipeline.add(match(and(conditions)));
+        }
+        pipeline.add(
+                group(getGroupingFields(USER, FIELD_DLAB_ID, DLAB_RESOURCE_TYPE, FIELD_PRODUCT, FIELD_RESOURCE_TYPE,
+                        FIELD_CURRENCY_CODE),
+                        sum(FIELD_COST, "$" + FIELD_COST),
+                        min(USAGE_FROM, "$" + FIELD_USAGE_DATE),
+                        max(USAGE_TO, "$" + FIELD_USAGE_DATE)
+                ));
+        pipeline.add(
+                sort(new Document(ID + "." + USER, 1)
+                        .append(ID + "." + FIELD_DLAB_ID, 1)
+                        .append(ID + "." + RESOURCE_TYPE, 1)
+                        .append(ID + "." + FIELD_PRODUCT, 1))
+        );
+
+        // Get billing report and the list of shape info
+        AggregateIterable<Document> agg = getCollection(BILLING).aggregate(pipeline);
+        Map<String, ShapeInfo> shapes = getShapes(filter.getShape());
+
+        // Build billing report lines
+        List<Document> reportItems = new ArrayList<>();
+        boolean filterByShape = !(filter.getShape() == null || filter.getShape().isEmpty());
+        String usageDateStart = null;
+        String usageDateEnd = null;
+        double costTotal = 0;
+
+        for (Document d : agg) {
+            Document id = (Document) d.get(ID);
+            String resourceId = id.getString(FIELD_DLAB_ID);
+            ShapeInfo shape = shapes.get(resourceId);
+            final UserInstanceStatus status = Optional.ofNullable(shape).map(ShapeInfo::getStatus).orElse(null);
+            if ((filterByShape && shape == null) || (!filter.getStatuses().isEmpty() && filter.getStatuses().stream()
+                    .noneMatch(s -> s.equals(status)))) {
+                continue;
+            }
+
+            String resourceTypeId = DlabResourceType.getResourceTypeName(id.getString(DLAB_RESOURCE_TYPE));
+            String shapeName = generateShapeName(shape);
+            String dateStart = d.getString(USAGE_FROM);
+            if (StringUtils.compare(usageDateStart, dateStart, false) > 0) {
+                usageDateStart = dateStart;
+            }
+            String dateEnd = d.getString(USAGE_TO);
+            if (StringUtils.compare(usageDateEnd, dateEnd) < 0) {
+                usageDateEnd = dateEnd;
+            }
+            double cost = BillingCalculationUtils.round(d.getDouble(FIELD_COST), 2);
+            costTotal += cost;
+
+            Document item = new Document()
+                    .append(FIELD_USER_ID, getUserOrDefault(id.getString(USER)))
+                    .append(FIELD_DLAB_ID, resourceId)
+                    .append(DLAB_RESOURCE_TYPE, resourceTypeId)
+                    .append(SHAPE, shapeName)
+                    .append(STATUS,
+                            Optional.ofNullable(status).map(UserInstanceStatus::toString).orElse(StringUtils.EMPTY))
+                    .append(FIELD_PRODUCT, id.getString(FIELD_PRODUCT))
+                    .append(FIELD_RESOURCE_TYPE, id.getString(FIELD_RESOURCE_TYPE))
+                    .append(FIELD_COST, BillingCalculationUtils.formatDouble(cost))
+                    .append(FIELD_CURRENCY_CODE, id.getString(FIELD_CURRENCY_CODE))
+                    .append(USAGE_FROM, dateStart)
+                    .append(USAGE_TO, dateEnd);
+            reportItems.add(item);
+        }
+
+        return new Document()
+                .append(SERVICE_BASE_NAME, settings.getServiceBaseName())
+                .append(TAG_RESOURCE_ID, settings.getConfTagResourceId())
+                .append(USAGE_FROM, usageDateStart)
+                .append(USAGE_TO, usageDateEnd)
+                .append(ITEMS, reportItems)
+                .append(COST_TOTAL, BillingCalculationUtils.formatDouble(BillingCalculationUtils.round(costTotal, 2)))
+                .append(FIELD_CURRENCY_CODE, (reportItems.isEmpty() ? null :
+                        reportItems.get(0).getString(FIELD_CURRENCY_CODE)))
+                .append(FULL_REPORT, isFullReport);
+    }
+
+    private void addCondition(List<Bson> conditions, String fieldName, List<String> values) {
+        if (values != null && !values.isEmpty()) {
+            conditions.add(in(fieldName, values));
+        }
+    }
 
-public class GcpBillingDao implements BillingDAO<BillingFilter> {
-	@Override
-	public Double getTotalCost() {
-		return null;
-	}
-
-	@Override
-	public Double getUserCost(String user) {
-		return null;
-	}
-
-	@Override
-	public int getBillingQuoteUsed() {
-		return 0;
-	}
-
-	@Override
-	public int getBillingUserQuoteUsed(String user) {
-		return 0;
-	}
-
-	@Override
-	public boolean isBillingQuoteReached() {
-		return false;
-	}
-
-	@Override
-	public boolean isUserQuoteReached(String user) {
-		return false;
-	}
-
-	@Override
-	public Document getReport(UserInfo userInfo, BillingFilter filter) {
-		return null;
-	}
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/modules/GcpSelfServiceModule.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/modules/GcpSelfServiceModule.java
index 03aee60..e48c26f 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/modules/GcpSelfServiceModule.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/modules/GcpSelfServiceModule.java
@@ -26,10 +26,13 @@ import com.epam.dlab.backendapi.dao.gcp.GcpBillingDao;
 import com.epam.dlab.backendapi.dao.gcp.GcpKeyDao;
 import com.epam.dlab.backendapi.resources.callback.gcp.EdgeCallbackGcp;
 import com.epam.dlab.backendapi.resources.callback.gcp.KeyUploaderCallbackGcp;
+import com.epam.dlab.backendapi.resources.gcp.BillingResourceGcp;
 import com.epam.dlab.backendapi.resources.gcp.ComputationalResourceGcp;
 import com.epam.dlab.backendapi.resources.gcp.GcpOauthResource;
+import com.epam.dlab.backendapi.service.BillingService;
 import com.epam.dlab.backendapi.service.InfrastructureInfoService;
 import com.epam.dlab.backendapi.service.InfrastructureTemplateService;
+import com.epam.dlab.backendapi.service.gcp.GcpBillingService;
 import com.epam.dlab.backendapi.service.gcp.GcpInfrastructureInfoService;
 import com.epam.dlab.backendapi.service.gcp.GcpInfrastructureTemplateService;
 import com.epam.dlab.cloud.CloudModule;
@@ -46,44 +49,46 @@ import org.quartz.impl.StdSchedulerFactory;
 
 public class GcpSelfServiceModule extends CloudModule {
 
-	private static final String MONGO_URI_FORMAT = "mongodb://%s:%s@%s:%d/%s";
-	private static final String QUARTZ_MONGO_URI_PROPERTY = "org.quartz.jobStore.mongoUri";
-	private static final String QUARTZ_DB_NAME = "org.quartz.jobStore.dbName";
+    private static final String MONGO_URI_FORMAT = "mongodb://%s:%s@%s:%d/%s";
+    private static final String QUARTZ_MONGO_URI_PROPERTY = "org.quartz.jobStore.mongoUri";
+    private static final String QUARTZ_DB_NAME = "org.quartz.jobStore.dbName";
 
-	@Override
-	@SuppressWarnings("unchecked")
-	public void init(Environment environment, Injector injector) {
+    @Override
+    @SuppressWarnings("unchecked")
+    public void init(Environment environment, Injector injector) {
 
-		environment.jersey().register(injector.getInstance(EdgeCallbackGcp.class));
-		environment.jersey().register(injector.getInstance(KeyUploaderCallbackGcp.class));
-		environment.jersey().register(injector.getInstance(ComputationalResourceGcp.class));
-		if (injector.getInstance(SelfServiceApplicationConfiguration.class).isGcpOuauth2AuthenticationEnabled()) {
-			environment.jersey().register(injector.getInstance(GcpOauthResource.class));
-		}
-		injector.getInstance(SecurityFactory.class).configure(injector, environment,
-				SelfServiceSecurityAuthenticator.class, injector.getInstance(Authorizer.class));
+        environment.jersey().register(injector.getInstance(EdgeCallbackGcp.class));
+        environment.jersey().register(injector.getInstance(KeyUploaderCallbackGcp.class));
+        environment.jersey().register(injector.getInstance(ComputationalResourceGcp.class));
+        environment.jersey().register(injector.getInstance(BillingResourceGcp.class));
+        if (injector.getInstance(SelfServiceApplicationConfiguration.class).isGcpOuauth2AuthenticationEnabled()) {
+            environment.jersey().register(injector.getInstance(GcpOauthResource.class));
+        }
+        injector.getInstance(SecurityFactory.class).configure(injector, environment,
+                SelfServiceSecurityAuthenticator.class, injector.getInstance(Authorizer.class));
 
-	}
+    }
 
-	@Override
-	protected void configure() {
-		bind((KeyDAO.class)).to(GcpKeyDao.class);
-		bind(InfrastructureInfoService.class).to(GcpInfrastructureInfoService.class);
-		bind(InfrastructureTemplateService.class).to(GcpInfrastructureTemplateService.class);
-		bind(SchedulerConfiguration.class).toInstance(
-				new SchedulerConfiguration(SelfServiceApplication.class.getPackage().getName()));
-		bind(BillingDAO.class).toInstance(new GcpBillingDao());
-	}
+    @Override
+    protected void configure() {
+        bind(BillingService.class).to(GcpBillingService.class);
+        bind((KeyDAO.class)).to(GcpKeyDao.class);
+        bind(InfrastructureInfoService.class).to(GcpInfrastructureInfoService.class);
+        bind(InfrastructureTemplateService.class).to(GcpInfrastructureTemplateService.class);
+        bind(BillingDAO.class).to(GcpBillingDao.class);
+        bind(SchedulerConfiguration.class).toInstance(
+                new SchedulerConfiguration(SelfServiceApplication.class.getPackage().getName()));
+    }
 
-	@Provides
-	@Singleton
-	Scheduler provideScheduler(SelfServiceApplicationConfiguration configuration) throws SchedulerException {
-		final MongoServiceFactory mongoFactory = configuration.getMongoFactory();
-		final String database = mongoFactory.getDatabase();
-		final String mongoUri = String.format(MONGO_URI_FORMAT, mongoFactory.getUsername(), mongoFactory.getPassword(),
-				mongoFactory.getHost(), mongoFactory.getPort(), database);
-		System.setProperty(QUARTZ_MONGO_URI_PROPERTY, mongoUri);
-		System.setProperty(QUARTZ_DB_NAME, database);
-		return StdSchedulerFactory.getDefaultScheduler();
-	}
+    @Provides
+    @Singleton
+    Scheduler provideScheduler(SelfServiceApplicationConfiguration configuration) throws SchedulerException {
+        final MongoServiceFactory mongoFactory = configuration.getMongoFactory();
+        final String database = mongoFactory.getDatabase();
+        final String mongoUri = String.format(MONGO_URI_FORMAT, mongoFactory.getUsername(), mongoFactory.getPassword(),
+                mongoFactory.getHost(), mongoFactory.getPort(), database);
+        System.setProperty(QUARTZ_MONGO_URI_PROPERTY, mongoUri);
+        System.setProperty(QUARTZ_DB_NAME, database);
+        return StdSchedulerFactory.getDefaultScheduler();
+    }
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/gcp/GcpBillingFilter.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/gcp/GcpBillingFilter.java
new file mode 100644
index 0000000..2966146
--- /dev/null
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/dto/gcp/GcpBillingFilter.java
@@ -0,0 +1,37 @@
+/*
+ * 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 com.epam.dlab.backendapi.resources.dto.gcp;
+
+import com.epam.dlab.backendapi.resources.dto.BillingFilter;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import lombok.Data;
+
+import java.util.List;
+
+@Data
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class GcpBillingFilter extends BillingFilter {
+
+    @JsonProperty
+    private List<String> shape;
+    @JsonProperty
+    private List<String> product;
+}
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/gcp/BillingResourceGcp.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/gcp/BillingResourceGcp.java
new file mode 100644
index 0000000..8218ba6
--- /dev/null
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/resources/gcp/BillingResourceGcp.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 com.epam.dlab.backendapi.resources.gcp;
+
+import com.epam.dlab.auth.UserInfo;
+import com.epam.dlab.backendapi.resources.dto.gcp.GcpBillingFilter;
+import com.epam.dlab.backendapi.service.BillingService;
+import com.google.inject.Inject;
+import io.dropwizard.auth.Auth;
+import org.bson.Document;
+
+import javax.validation.Valid;
+import javax.validation.constraints.NotNull;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+@Path("/billing")
+public class BillingResourceGcp {
+
+    @Inject
+    private BillingService billingService;
+
+    /**
+     * Returns the billing report.
+     *
+     * @param userInfo user info.
+     * @param formDTO  filter for report data.
+     */
+    @POST
+    @Path("/report")
+    @Produces(MediaType.APPLICATION_JSON)
+    @SuppressWarnings("unchecked")
+    public Document getBillingReport(@Auth UserInfo userInfo, @Valid @NotNull GcpBillingFilter formDTO) {
+        return billingService.getBillingReport(userInfo, formDTO);
+    }
+
+}
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/service/BillingService.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/service/BillingService.java
index f9098f8..cd747e6 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/service/BillingService.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/service/BillingService.java
@@ -18,66 +18,81 @@ package com.epam.dlab.backendapi.service;
 
 import com.epam.dlab.auth.UserInfo;
 import com.epam.dlab.backendapi.dao.BaseBillingDAO;
+import com.epam.dlab.backendapi.dao.BillingDAO;
 import com.epam.dlab.backendapi.resources.dto.BillingFilter;
 import com.epam.dlab.backendapi.util.CSVFormatter;
 import com.epam.dlab.exceptions.DlabException;
+import com.google.inject.Inject;
 import jersey.repackaged.com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
 import org.bson.Document;
 
 import java.text.ParseException;
 import java.util.List;
 
+@Slf4j
 public abstract class BillingService<T extends BillingFilter> {
 
-	protected abstract Document getReport(UserInfo userInfo, T filter);
+    @Inject
+    private BillingDAO billingDAO;
 
-	protected String getValueOrEmpty(Document document, String key) {
-		String value = document.getString(key);
-		return value == null ? "" : value;
-	}
+    public Document getReport(UserInfo userInfo, T filter) {
+        log.trace("Get billing report for user {} with filter {}", userInfo.getName(), filter);
+        try {
+            return billingDAO.getReport(userInfo, filter);
+        } catch (RuntimeException t) {
+            log.error("Cannot load billing report for user {} with filter {}", userInfo.getName(), filter, t);
+            throw new DlabException("Cannot load billing report: " + t.getLocalizedMessage(), t);
+        }
+    }
 
-	String getHeaders(boolean full) {
-		return CSVFormatter.formatLine(getHeadersList(full), CSVFormatter.SEPARATOR);
-	}
+    protected String getValueOrEmpty(Document document, String key) {
+        String value = document.getString(key);
+        return value == null ? "" : value;
+    }
 
-	public Document getBillingReport(UserInfo userInfo, T filter) {
-		filter.getUser().replaceAll(s -> s.equalsIgnoreCase(BaseBillingDAO.SHARED_RESOURCE_NAME) ? null : s);
-		return getReport(userInfo, filter);
-	}
+    String getHeaders(boolean full) {
+        return CSVFormatter.formatLine(getHeadersList(full), CSVFormatter.SEPARATOR);
+    }
 
-	public byte[] downloadReport(UserInfo userInfo, T filter) {
-		return prepareReport(getReport(userInfo, filter)).getBytes();
-	}
+    public Document getBillingReport(UserInfo userInfo, T filter) {
+        filter.getUser().replaceAll(s -> s.equalsIgnoreCase(BaseBillingDAO.SHARED_RESOURCE_NAME) ? null : s);
+        return getReport(userInfo, filter);
+    }
 
-	String prepareReport(Document document) {
-		try {
-			StringBuilder builder =
-					new StringBuilder(CSVFormatter.formatLine(Lists.newArrayList(getFirstLine(document)),
-							CSVFormatter.SEPARATOR, '\"'));
+    public byte[] downloadReport(UserInfo userInfo, T filter) {
+        return prepareReport(getReport(userInfo, filter)).getBytes();
+    }
 
-			Boolean full = (Boolean) document.get(BaseBillingDAO.FULL_REPORT);
-			builder.append(getHeaders(full));
+    String prepareReport(Document document) {
+        try {
+            StringBuilder builder =
+                    new StringBuilder(CSVFormatter.formatLine(Lists.newArrayList(getFirstLine(document)),
+                            CSVFormatter.SEPARATOR, '\"'));
 
-			@SuppressWarnings("unchecked")
-			List<Document> items = (List<Document>) document.get(BaseBillingDAO.ITEMS);
+            Boolean full = (Boolean) document.get(BaseBillingDAO.FULL_REPORT);
+            builder.append(getHeaders(full));
 
-			items.forEach(d -> builder.append(getLine(full, d)));
+            @SuppressWarnings("unchecked")
+            List<Document> items = (List<Document>) document.get(BaseBillingDAO.ITEMS);
 
-			builder.append(getTotal(full, document));
+            items.forEach(d -> builder.append(getLine(full, d)));
 
-			return builder.toString();
-		} catch (ParseException e) {
-			throw new DlabException("Cannot prepare CSV file", e);
-		}
-	}
+            builder.append(getTotal(full, document));
 
-	public abstract String getFirstLine(Document document) throws ParseException;
+            return builder.toString();
+        } catch (ParseException e) {
+            throw new DlabException("Cannot prepare CSV file", e);
+        }
+    }
 
-	public abstract List<String> getHeadersList(boolean full);
+    public abstract String getFirstLine(Document document) throws ParseException;
 
-	public abstract String getLine(boolean full, Document document);
+    public abstract List<String> getHeadersList(boolean full);
 
-	public abstract String getTotal(boolean full, Document document);
+    public abstract String getLine(boolean full, Document document);
 
-	public abstract String getReportFileName(UserInfo userInfo, T filter);
+    public abstract String getTotal(boolean full, Document document);
+
+    public abstract String getReportFileName(UserInfo userInfo, T filter);
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/service/aws/AwsBillingService.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/service/aws/AwsBillingService.java
index ca19a3f..c89311b 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/service/aws/AwsBillingService.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/service/aws/AwsBillingService.java
@@ -38,20 +38,6 @@ import java.util.List;
 @Singleton
 public class AwsBillingService extends BillingService<AwsBillingFilter> {
 
-	@Inject
-	private BillingDAO billingDAO;
-
-	@Override
-	protected Document getReport(UserInfo userInfo, AwsBillingFilter filter) {
-		log.trace("Get billing report for user {} with filter {}", userInfo.getName(), filter);
-		try {
-			return billingDAO.getReport(userInfo, filter);
-		} catch (RuntimeException t) {
-			log.error("Cannot load billing report for user {} with filter {}", userInfo.getName(), filter, t);
-			throw new DlabException("Cannot load billing report: " + t.getLocalizedMessage(), t);
-		}
-	}
-
 	@Override
 	public String getReportFileName(UserInfo userInfo, AwsBillingFilter filter) {
 		return "aws-billing-report.csv";
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/service/azure/AzureBillingService.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/service/azure/AzureBillingService.java
index 7a684f7..cac4b41 100644
--- a/services/self-service/src/main/java/com/epam/dlab/backendapi/service/azure/AzureBillingService.java
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/service/azure/AzureBillingService.java
@@ -24,7 +24,6 @@ import com.epam.dlab.backendapi.dao.azure.AzureBillingDAO;
 import com.epam.dlab.backendapi.resources.dto.azure.AzureBillingFilter;
 import com.epam.dlab.backendapi.service.BillingService;
 import com.epam.dlab.backendapi.util.CSVFormatter;
-import com.epam.dlab.exceptions.DlabException;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import lombok.extern.slf4j.Slf4j;
@@ -39,84 +38,73 @@ import java.util.List;
 @Singleton
 public class AzureBillingService extends BillingService<AzureBillingFilter> {
 
-	@Inject
-	private BillingDAO billingDAO;
-
-	@Override
-	public Document getReport(UserInfo userInfo, AzureBillingFilter filter) {
-		log.trace("Get billing report for user {} with filter {}", userInfo.getName(), filter);
-		try {
-			return billingDAO.getReport(userInfo, filter);
-		} catch (RuntimeException t) {
-			log.error("Cannot load billing report for user {} with filter {}", userInfo.getName(), filter, t);
-			throw new DlabException("Cannot load billing report: " + t.getLocalizedMessage(), t);
-		}
-	}
-
-	@Override
-	public String getReportFileName(UserInfo userInfo, AzureBillingFilter filter) {
-		return "azure-billing-report.csv";
-	}
-
-	@Override
-	public String getFirstLine(Document document) throws ParseException {
-		SimpleDateFormat from = new SimpleDateFormat("yyyy-MM-dd");
-		SimpleDateFormat to = new SimpleDateFormat("MMM dd, yyyy");
-
-		return String.format("Service base name: %s  " +
-						"Available reporting period from: %s to: %s",
-				document.get(BaseBillingDAO.SERVICE_BASE_NAME),
-				to.format(from.parse((String) document.get(MongoKeyWords.USAGE_FROM))),
-				to.format(from.parse((String) document.get(MongoKeyWords.USAGE_TO))));
-	}
-
-	public List<String> getHeadersList(boolean full) {
-		List<String> headers = new ArrayList<>();
-
-		if (full) {
-			headers.add("USER");
-		}
-
-		headers.add("ENVIRONMENT NAME");
-		headers.add("RESOURCE TYPE");
-		headers.add("INSTANCE SIZE");
-		headers.add("CATEGORY");
-		headers.add("SERVICE CHARGES");
-
-		return headers;
-	}
-
-	@Override
-	public String getLine(boolean full, Document document) {
-		List<String> items = new ArrayList<>();
-
-		if (full) {
-			items.add(getValueOrEmpty(document, MongoKeyWords.DLAB_USER));
-		}
-
-		items.add(getValueOrEmpty(document, MongoKeyWords.DLAB_ID));
-		items.add(getValueOrEmpty(document, MongoKeyWords.RESOURCE_TYPE));
-		items.add(getValueOrEmpty(document, AzureBillingDAO.SIZE).replace(System.lineSeparator(), " "));
-		items.add(getValueOrEmpty(document, MongoKeyWords.METER_CATEGORY));
-
-		items.add(getValueOrEmpty(document, MongoKeyWords.COST_STRING)
-				+ " " + getValueOrEmpty(document, MongoKeyWords.CURRENCY_CODE));
-
-		return CSVFormatter.formatLine(items, CSVFormatter.SEPARATOR);
-	}
-
-	@Override
-	public String getTotal(boolean full, Document document) {
-		int padding = getHeadersList(full).size() - 1;
-
-		List<String> items = new ArrayList<>();
-		while (padding-- > 0) {
-			items.add("");
-		}
-
-		items.add(String.format("Total: %s %s", getValueOrEmpty(document, MongoKeyWords.COST_STRING),
-				getValueOrEmpty(document, MongoKeyWords.CURRENCY_CODE)));
-
-		return CSVFormatter.formatLine(items, CSVFormatter.SEPARATOR);
-	}
+    @Inject
+    private BillingDAO billingDAO;
+
+    @Override
+    public String getReportFileName(UserInfo userInfo, AzureBillingFilter filter) {
+        return "azure-billing-report.csv";
+    }
+
+    @Override
+    public String getFirstLine(Document document) throws ParseException {
+        SimpleDateFormat from = new SimpleDateFormat("yyyy-MM-dd");
+        SimpleDateFormat to = new SimpleDateFormat("MMM dd, yyyy");
+
+        return String.format("Service base name: %s  " +
+                        "Available reporting period from: %s to: %s",
+                document.get(BaseBillingDAO.SERVICE_BASE_NAME),
+                to.format(from.parse((String) document.get(MongoKeyWords.USAGE_FROM))),
+                to.format(from.parse((String) document.get(MongoKeyWords.USAGE_TO))));
+    }
+
+    public List<String> getHeadersList(boolean full) {
+        List<String> headers = new ArrayList<>();
+
+        if (full) {
+            headers.add("USER");
+        }
+
+        headers.add("ENVIRONMENT NAME");
+        headers.add("RESOURCE TYPE");
+        headers.add("INSTANCE SIZE");
+        headers.add("CATEGORY");
+        headers.add("SERVICE CHARGES");
+
+        return headers;
+    }
+
+    @Override
+    public String getLine(boolean full, Document document) {
+        List<String> items = new ArrayList<>();
+
+        if (full) {
+            items.add(getValueOrEmpty(document, MongoKeyWords.DLAB_USER));
+        }
+
+        items.add(getValueOrEmpty(document, MongoKeyWords.DLAB_ID));
+        items.add(getValueOrEmpty(document, MongoKeyWords.RESOURCE_TYPE));
+        items.add(getValueOrEmpty(document, AzureBillingDAO.SIZE).replace(System.lineSeparator(), " "));
+        items.add(getValueOrEmpty(document, MongoKeyWords.METER_CATEGORY));
+
+        items.add(getValueOrEmpty(document, MongoKeyWords.COST_STRING)
+                + " " + getValueOrEmpty(document, MongoKeyWords.CURRENCY_CODE));
+
+        return CSVFormatter.formatLine(items, CSVFormatter.SEPARATOR);
+    }
+
+    @Override
+    public String getTotal(boolean full, Document document) {
+        int padding = getHeadersList(full).size() - 1;
+
+        List<String> items = new ArrayList<>();
+        while (padding-- > 0) {
+            items.add("");
+        }
+
+        items.add(String.format("Total: %s %s", getValueOrEmpty(document, MongoKeyWords.COST_STRING),
+                getValueOrEmpty(document, MongoKeyWords.CURRENCY_CODE)));
+
+        return CSVFormatter.formatLine(items, CSVFormatter.SEPARATOR);
+    }
 }
diff --git a/services/self-service/src/main/java/com/epam/dlab/backendapi/service/gcp/GcpBillingService.java b/services/self-service/src/main/java/com/epam/dlab/backendapi/service/gcp/GcpBillingService.java
new file mode 100644
index 0000000..42d2095
--- /dev/null
+++ b/services/self-service/src/main/java/com/epam/dlab/backendapi/service/gcp/GcpBillingService.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 com.epam.dlab.backendapi.service.gcp;
+
+import com.epam.dlab.auth.UserInfo;
+import com.epam.dlab.backendapi.resources.dto.gcp.GcpBillingFilter;
+import com.epam.dlab.backendapi.service.BillingService;
+import org.bson.Document;
+
+import java.text.ParseException;
+import java.util.List;
+
+public class GcpBillingService extends BillingService<GcpBillingFilter> {
+    @Override
+    public String getFirstLine(Document document) throws ParseException {
+        return null;
+    }
+
+    @Override
+    public List<String> getHeadersList(boolean full) {
+        return null;
+    }
+
+    @Override
+    public String getLine(boolean full, Document document) {
+        return null;
+    }
+
+    @Override
+    public String getTotal(boolean full, Document document) {
+        return null;
+    }
+
+    @Override
+    public String getReportFileName(UserInfo userInfo, GcpBillingFilter filter) {
+        return null;
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@dlab.apache.org
For additional commands, e-mail: commits-help@dlab.apache.org