Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-3237. Recon should provide the list of datanodes that a missing … #753

Merged
merged 3 commits into from
Apr 4, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions hadoop-ozone/dist/src/main/smoketest/recon/recon-api.robot
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,9 @@ Check if Recon picks up DN heartbeats
Should contain ${result} \"healthyDatanodes\":3
Should contain ${result} \"pipelines\":4

${result} = Execute curl --negotiate -u : -v ${API_ENDPOINT_URL}/containers/1/replicaHistory
Should contain ${result} \"containerId\":1

Check if Recon Web UI is up
Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit HTTP user
${result} = Execute curl --negotiate -u : -v ${ENDPOINT_URL}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ static void cleanup() {
public static void main(String[] args) {
if (args.length < 1) {
throw new IllegalArgumentException("Missing required arguments: " +
"Need a ouput directory for generated code.\nUsage: " +
"Need an output directory for generated code.\nUsage: " +
"org.apache.hadoop.ozone.recon.persistence.JooqCodeGenerator " +
"<outputDirectory>.");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
*/
package org.hadoop.ozone.recon.codegen;

import org.hadoop.ozone.recon.schema.ContainerSchemaDefinition;
import org.hadoop.ozone.recon.schema.ReconTaskSchemaDefinition;
import org.hadoop.ozone.recon.schema.ReconSchemaDefinition;
import org.hadoop.ozone.recon.schema.StatsSchemaDefinition;
Expand All @@ -36,6 +37,7 @@ protected void configure() {
Multibinder<ReconSchemaDefinition> schemaBinder =
Multibinder.newSetBinder(binder(), ReconSchemaDefinition.class);
schemaBinder.addBinding().to(UtilizationSchemaDefinition.class);
schemaBinder.addBinding().to(ContainerSchemaDefinition.class);
schemaBinder.addBinding().to(ReconTaskSchemaDefinition.class);
schemaBinder.addBinding().to(StatsSchemaDefinition.class);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.hadoop.ozone.recon.schema;

import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.jooq.DSLContext;
import org.jooq.impl.DSL;
import org.jooq.impl.SQLDataType;

import javax.sql.DataSource;
import java.sql.Connection;
import java.sql.SQLException;

/**
* Class used to create tables that are required for tracking containers.
*/
@Singleton
public class ContainerSchemaDefinition implements ReconSchemaDefinition {

public static final String CONTAINER_HISTORY_TABLE_NAME =
"container_history";
public static final String MISSING_CONTAINERS_TABLE_NAME =
"missing_containers";
private static final String CONTAINER_ID = "container_id";
private final DataSource dataSource;
private DSLContext dslContext;

@Inject
ContainerSchemaDefinition(DataSource dataSource) {
this.dataSource = dataSource;
}

@Override
public void initializeSchema() throws SQLException {
Connection conn = dataSource.getConnection();
dslContext = DSL.using(conn);
createContainerHistoryTable();
createMissingContainersTable();
}

/**
* Create the Container History table.
*/
private void createContainerHistoryTable() {
dslContext.createTableIfNotExists(CONTAINER_HISTORY_TABLE_NAME)
.column(CONTAINER_ID, SQLDataType.BIGINT)
.column("datanode_host", SQLDataType.VARCHAR(1024))
.column("first_report_timestamp", SQLDataType.BIGINT)
.column("last_report_timestamp", SQLDataType.BIGINT)
.constraint(DSL.constraint("pk_container_id_datanode_host")
.primaryKey(CONTAINER_ID, "datanode_host"))
.execute();
}

/**
* Create the Missing Containers table.
*/
private void createMissingContainersTable() {
dslContext.createTableIfNotExists(MISSING_CONTAINERS_TABLE_NAME)
.column(CONTAINER_ID, SQLDataType.BIGINT)
.column("missing_since", SQLDataType.BIGINT)
.constraint(DSL.constraint("pk_container_id")
.primaryKey(CONTAINER_ID))
.execute();
}

public DSLContext getDSLContext() {
avijayanhwx marked this conversation as resolved.
Show resolved Hide resolved
return dslContext;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@

import javax.sql.DataSource;

import com.google.inject.Singleton;
import org.jooq.impl.DSL;
import org.jooq.impl.SQLDataType;

Expand All @@ -32,6 +33,7 @@
* Class used to create tables that are required for Recon's task
* management.
*/
@Singleton
public class ReconTaskSchemaDefinition implements ReconSchemaDefinition {

public static final String RECON_TASK_STATUS_TABLE_NAME =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.hadoop.ozone.recon.schema;

import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.jooq.impl.DSL;
import org.jooq.impl.SQLDataType;

Expand All @@ -29,6 +30,7 @@
/**
* Class used to create tables that are required for storing Ozone statistics.
*/
@Singleton
public class StatsSchemaDefinition implements ReconSchemaDefinition {

public static final String GLOBAL_STATS_TABLE_NAME = "global_stats";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@

import javax.sql.DataSource;

import com.google.inject.Singleton;
import org.jooq.impl.DSL;
import org.jooq.impl.SQLDataType;
import org.springframework.transaction.annotation.Transactional;
Expand All @@ -31,6 +32,7 @@
/**
* Programmatic definition of Recon DDL.
*/
@Singleton
public class UtilizationSchemaDefinition implements ReconSchemaDefinition {

private final DataSource dataSource;
Expand All @@ -41,9 +43,6 @@ public class UtilizationSchemaDefinition implements ReconSchemaDefinition {
public static final String FILE_COUNT_BY_SIZE_TABLE_NAME =
"file_count_by_size";

public static final String MISSING_CONTAINERS_TABLE_NAME =
"missing_containers";

@Inject
UtilizationSchemaDefinition(DataSource dataSource) {
this.dataSource = dataSource;
Expand All @@ -55,10 +54,9 @@ public void initializeSchema() throws SQLException {
Connection conn = dataSource.getConnection();
createClusterGrowthTable(conn);
createFileSizeCount(conn);
createMissingContainersTable(conn);
}

void createClusterGrowthTable(Connection conn) {
private void createClusterGrowthTable(Connection conn) {
DSL.using(conn).createTableIfNotExists(CLUSTER_GROWTH_DAILY_TABLE_NAME)
.column("timestamp", SQLDataType.TIMESTAMP)
.column("datanode_id", SQLDataType.INTEGER)
Expand All @@ -73,21 +71,12 @@ void createClusterGrowthTable(Connection conn) {
.execute();
}

void createFileSizeCount(Connection conn) {
private void createFileSizeCount(Connection conn) {
DSL.using(conn).createTableIfNotExists(FILE_COUNT_BY_SIZE_TABLE_NAME)
.column("file_size", SQLDataType.BIGINT)
.column("count", SQLDataType.BIGINT)
.constraint(DSL.constraint("pk_file_size")
.primaryKey("file_size"))
.execute();
}

void createMissingContainersTable(Connection conn) {
DSL.using(conn).createTableIfNotExists(MISSING_CONTAINERS_TABLE_NAME)
.column("container_id", SQLDataType.BIGINT)
.column("missing_since", SQLDataType.BIGINT)
.constraint(DSL.constraint("pk_container_id")
.primaryKey("container_id"))
.execute();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.hadoop.ozone.om.OMMetadataManager;
import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
import org.apache.hadoop.ozone.recon.persistence.ContainerSchemaManager;
import org.apache.hadoop.ozone.recon.persistence.DataSourceConfiguration;
import org.apache.hadoop.ozone.recon.persistence.JooqPersistenceModule;
import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
Expand All @@ -59,6 +60,7 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.hdds.utils.db.DBStore;
import org.apache.ratis.protocol.ClientId;
import org.hadoop.ozone.recon.schema.tables.daos.ContainerHistoryDao;
import org.hadoop.ozone.recon.schema.tables.daos.FileCountBySizeDao;
import org.hadoop.ozone.recon.schema.tables.daos.MissingContainersDao;
import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao;
Expand Down Expand Up @@ -88,6 +90,7 @@ protected void configure() {
.to(ReconOmMetadataManagerImpl.class);
bind(OMMetadataManager.class).to(ReconOmMetadataManagerImpl.class);

bind(ContainerSchemaManager.class).in(Singleton.class);
bind(ContainerDBServiceProvider.class)
.to(ContainerDBServiceProviderImpl.class).in(Singleton.class);
bind(OzoneManagerServiceProvider.class)
Expand Down Expand Up @@ -117,6 +120,11 @@ MissingContainersDao getMissingContainersDao(final Configuration sqlConfig) {
return new MissingContainersDao(sqlConfig);
}

@Provides
ContainerHistoryDao getContainerHistoryDao(final Configuration sqlConfig) {
return new ContainerHistoryDao(sqlConfig);
}

@Provides
FileCountBySizeDao getFileCountBySizeDao(final Configuration sqlConfig) {
return new FileCountBySizeDao(sqlConfig);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import java.io.IOException;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -53,9 +52,11 @@
import org.apache.hadoop.ozone.recon.api.types.KeysResponse;
import org.apache.hadoop.ozone.recon.api.types.MissingContainerMetadata;
import org.apache.hadoop.ozone.recon.api.types.MissingContainersResponse;
import org.apache.hadoop.ozone.recon.persistence.ContainerSchemaManager;
import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
import org.apache.hadoop.ozone.recon.scm.ReconContainerManager;
import org.apache.hadoop.ozone.recon.spi.ContainerDBServiceProvider;
import org.hadoop.ozone.recon.schema.tables.pojos.ContainerHistory;

import static org.apache.hadoop.ozone.recon.ReconConstants.DEFAULT_FETCH_COUNT;
import static org.apache.hadoop.ozone.recon.ReconConstants.PREV_CONTAINER_ID_DEFAULT_VALUE;
Expand All @@ -77,11 +78,14 @@ public class ContainerEndpoint {
private ReconOMMetadataManager omMetadataManager;

private ReconContainerManager containerManager;
private ContainerSchemaManager containerSchemaManager;

@Inject
public ContainerEndpoint(OzoneStorageContainerManager reconSCM) {
public ContainerEndpoint(OzoneStorageContainerManager reconSCM,
ContainerSchemaManager containerSchemaManager) {
this.containerManager =
(ReconContainerManager) reconSCM.getContainerManager();
this.containerSchemaManager = containerSchemaManager;
}

/**
Expand Down Expand Up @@ -203,6 +207,21 @@ public Response getKeysForContainer(
return Response.ok(keysResponse).build();
}

/**
* Return Container replica history for the container identified by the id
* param.
*
* @param containerID the given containerID.
* @return {@link Response}
*/
@GET
@Path("/{id}/replicaHistory")
public Response getReplicaHistoryForContainer(
@PathParam("id") Long containerID) {
return Response.ok(
containerSchemaManager.getAllContainerHistory(containerID)).build();
}

/**
* Return
* {@link org.apache.hadoop.ozone.recon.api.types.MissingContainerMetadata}
Expand All @@ -222,9 +241,9 @@ public Response getMissingContainers() {
long keyCount = containerInfo.getNumberOfKeys();
UUID pipelineID = containerInfo.getPipelineID().getId();

// TODO: Find out which datanodes had replicas of this container
// and populate this list
List datanodes = Collections.emptyList();
List<ContainerHistory> datanodes =
containerSchemaManager.getLatestContainerHistory(
containerID, containerInfo.getReplicationFactor().getNumber());
missingContainers.add(new MissingContainerMetadata(containerID,
container.getMissingSince(), keyCount, pipelineID, datanodes));
} catch (IOException ioEx) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
*/
package org.apache.hadoop.ozone.recon.api.types;

import org.hadoop.ozone.recon.schema.tables.pojos.ContainerHistory;

import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlElement;
Expand All @@ -41,17 +43,17 @@ public class MissingContainerMetadata {
@XmlElement(name = "pipelineID")
private UUID pipelineID;

@XmlElement(name = "datanodes")
private List<String> datanodes;
@XmlElement(name = "replicas")
private List<ContainerHistory> replicas;

public MissingContainerMetadata(long containerID, long missingSince,
long keys, UUID pipelineID,
List<String> datanodes) {
List<ContainerHistory> replicas) {
this.containerID = containerID;
this.missingSince = missingSince;
this.keys = keys;
this.pipelineID = pipelineID;
this.datanodes = datanodes;
this.replicas = replicas;
}

public long getContainerID() {
Expand All @@ -62,8 +64,8 @@ public long getKeys() {
return keys;
}

public List<String> getDatanodes() {
return datanodes;
public List<ContainerHistory> getReplicas() {
return replicas;
}

public long getMissingSince() {
Expand Down
Loading