You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by jd...@apache.org on 2015/06/23 12:25:11 UTC

[23/51] [abbrv] incubator-lens git commit: LENS-531 : Add cli command to show all queryable fields of cube or dimension

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/ff31ad96/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java
index 208eb78..a22862e 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java
@@ -18,11 +18,18 @@
  */
 package org.apache.lens.cli;
 
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+
 import java.io.*;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.util.Arrays;
 
+import org.apache.lens.api.metastore.XJoinChains;
 import org.apache.lens.cli.commands.LensDimensionCommands;
+import org.apache.lens.cli.table.XJoinChainTable;
 import org.apache.lens.client.LensClient;
 
 import org.testng.Assert;
@@ -74,13 +81,26 @@ public class TestLensDimensionCommands extends LensCliApplicationTest {
     createDimension();
     dimensionList = getCommand().showDimensions();
     Assert.assertTrue(dimensionList.contains("test_dim"));
-
+    testFields(getCommand());
+    testJoinChains(getCommand());
     testUpdateCommand(new File(dimensionSpec.toURI()), getCommand());
     getCommand().dropDimension("test_dim");
     dimensionList = getCommand().showDimensions();
     Assert.assertFalse(dimensionList.contains("test_dim"));
   }
 
+  private void testJoinChains(LensDimensionCommands command) {
+    assertEquals(command.showJoinChains("test_dim"), new XJoinChainTable(new XJoinChains()).toString());
+  }
+
+  private void testFields(LensDimensionCommands qCom) {
+    String testDimFields = qCom.showQueryableFields("test_dim", true);
+    for (String field : Arrays.asList("detail", "id", "d2id", "name")) {
+      assertTrue(testDimFields.contains(field));
+    }
+    assertFalse(testDimFields.contains("measure"));
+  }
+
   /**
    * Test update command.
    *
@@ -103,8 +123,8 @@ public class TestLensDimensionCommands extends LensCliApplicationTest {
       String xmlContent = sb.toString();
 
       xmlContent = xmlContent.replace("<property name=\"test_dim.prop\" value=\"test\" />\n",
-          "<property name=\"test_dim.prop\" value=\"test\" />"
-              + "\n<property name=\"test_dim.prop1\" value=\"test1\" />\n");
+        "<property name=\"test_dim.prop\" value=\"test\" />"
+          + "\n<property name=\"test_dim.prop1\" value=\"test1\" />\n");
 
       File newFile = new File("/tmp/test_dim1.xml");
       Writer writer = new OutputStreamWriter(new FileOutputStream(newFile));

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/ff31ad96/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
index d5fd999..97f0cf0 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
@@ -20,20 +20,17 @@ package org.apache.lens.cli;
 
 import static org.testng.Assert.*;
 
-import java.io.*;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.net.URL;
-import java.util.GregorianCalendar;
-import java.util.Scanner;
-import java.util.UUID;
+import java.util.*;
 
 import javax.ws.rs.BadRequestException;
 import javax.xml.datatype.DatatypeFactory;
 
 import org.apache.lens.api.APIResult;
-import org.apache.lens.api.metastore.XPartition;
-import org.apache.lens.api.metastore.XTimePartSpec;
-import org.apache.lens.api.metastore.XTimePartSpecElement;
-import org.apache.lens.api.metastore.XUpdatePeriod;
+import org.apache.lens.api.metastore.*;
 import org.apache.lens.api.query.QueryHandle;
 import org.apache.lens.cli.commands.LensCubeCommands;
 import org.apache.lens.cli.commands.LensQueryCommands;
@@ -108,6 +105,7 @@ public class TestLensQueryCommands extends LensCliApplicationTest {
     testFailPreparedQuery(qCom);
   }
 
+
   /**
    * Test prepared query.
    *
@@ -303,6 +301,7 @@ public class TestLensQueryCommands extends LensCliApplicationTest {
     assertTrue(qCom.getQueryResults(qh, resDir.getAbsolutePath(), true).contains("Saved"));
     assertEquals(readFile(resDir.getAbsolutePath() + File.separator + qh + ".csv").trim(), expected.trim());
   }
+
   private String readFile(String path) throws FileNotFoundException {
     return new Scanner(new File(path)).useDelimiter("\\Z").next();
   }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/ff31ad96/lens-client/src/main/java/org/apache/lens/client/LensClient.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/LensClient.java b/lens-client/src/main/java/org/apache/lens/client/LensClient.java
index 7c82619..ca017b8 100644
--- a/lens-client/src/main/java/org/apache/lens/client/LensClient.java
+++ b/lens-client/src/main/java/org/apache/lens/client/LensClient.java
@@ -397,6 +397,13 @@ public class LensClient {
     return mc.getCube(cubeName);
   }
 
+  public XFlattenedColumns getQueryableFields(String table, boolean flattened) {
+    return mc.getQueryableFields(table, flattened);
+  }
+  public XJoinChains getJoinChains(String table) {
+    return mc.getJoinChains(table);
+  }
+
   public XDimension getDimension(String dimName) {
     return mc.getDimension(dimName);
   }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/ff31ad96/lens-client/src/main/java/org/apache/lens/client/LensMetadataClient.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/LensMetadataClient.java b/lens-client/src/main/java/org/apache/lens/client/LensMetadataClient.java
index 44f4f1f..c58f5ee 100644
--- a/lens-client/src/main/java/org/apache/lens/client/LensMetadataClient.java
+++ b/lens-client/src/main/java/org/apache/lens/client/LensMetadataClient.java
@@ -28,17 +28,11 @@ import java.nio.charset.Charset;
 import java.util.Date;
 import java.util.List;
 
-import javax.ws.rs.client.Client;
-import javax.ws.rs.client.ClientBuilder;
-import javax.ws.rs.client.Entity;
-import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.client.*;
 import javax.ws.rs.core.GenericType;
 import javax.ws.rs.core.MediaType;
 import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
+import javax.xml.bind.*;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
@@ -51,11 +45,7 @@ import org.apache.lens.api.metastore.*;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import org.glassfish.jersey.media.multipart.FormDataBodyPart;
-import org.glassfish.jersey.media.multipart.FormDataContentDisposition;
-import org.glassfish.jersey.media.multipart.FormDataMultiPart;
-import org.glassfish.jersey.media.multipart.MultiPartFeature;
-
+import org.glassfish.jersey.media.multipart.*;
 import org.xml.sax.SAXException;
 
 import com.google.common.base.Joiner;
@@ -256,6 +246,25 @@ public class LensMetadataClient {
     return cube.getValue();
   }
 
+  public XFlattenedColumns getQueryableFields(String tableName, boolean flattened) {
+    WebTarget target = getMetastoreWebTarget();
+    JAXBElement<XFlattenedColumns> fields = target.path("flattened").path(tableName)
+      .queryParam("sessionid", this.connection.getSessionHandle())
+      .queryParam("add_chains", flattened)
+      .request(MediaType.APPLICATION_XML).get(new GenericType<JAXBElement<XFlattenedColumns>>() {
+      });
+    return fields.getValue();
+  }
+
+  public XJoinChains getJoinChains(String tableName) {
+    WebTarget target = getMetastoreWebTarget();
+    JAXBElement<XJoinChains> fields = target.path("chains").path(tableName)
+      .queryParam("sessionid", this.connection.getSessionHandle())
+      .request(MediaType.APPLICATION_XML).get(new GenericType<JAXBElement<XJoinChains>>() {
+      });
+    return fields.getValue();
+  }
+
   public APIResult dropCube(String cubeName) {
     WebTarget target = getMetastoreWebTarget();
     APIResult result = target.path("cubes").path(cubeName)

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/ff31ad96/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java b/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
index 890da4c..e0aa47c 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
@@ -534,10 +534,12 @@ public interface CubeMetastoreService {
    *
    * @param sessionHandle The session handle
    * @param tableName     The table name - cube name or dimension name
+   * @param addChains
    * @return {@link XFlattenedColumns}
    * @throws LensException
    */
-  XFlattenedColumns getFlattenedColumns(LensSessionHandle sessionHandle, String tableName) throws LensException;
+  XFlattenedColumns getFlattenedColumns(LensSessionHandle sessionHandle, String tableName, boolean addChains)
+    throws LensException;
 
   /**
    * Get the latest available date upto which data is available for the base cubes, for the time dimension
@@ -553,4 +555,6 @@ public interface CubeMetastoreService {
 
   List<String> getPartitionTimelines(LensSessionHandle sessionid, String factName, String storage,
     String updatePeriod, String timeDimension) throws LensException, HiveException;
+
+  XJoinChains getAllJoinChains(LensSessionHandle sessionid, String table) throws LensException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/ff31ad96/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
index 64f6cd4..07e20dc 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
@@ -26,6 +26,7 @@ import javax.ws.rs.NotFoundException;
 import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.api.metastore.*;
 import org.apache.lens.cube.metadata.*;
+import org.apache.lens.cube.metadata.Dimension;
 import org.apache.lens.cube.metadata.timeline.PartitionTimeline;
 import org.apache.lens.server.LensService;
 import org.apache.lens.server.api.error.LensException;
@@ -34,8 +35,7 @@ import org.apache.lens.server.session.LensSessionImpl;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.metastore.api.*;
-import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.*;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.ParseException;
@@ -44,7 +44,6 @@ import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.thrift.TException;
 
 import com.google.common.collect.Lists;
-
 import lombok.extern.slf4j.Slf4j;
 
 @Slf4j
@@ -1282,14 +1281,31 @@ public class CubeMetastoreServiceImpl extends LensService implements CubeMetasto
 
   private void addAllDirectExpressionsToFlattenedList(ObjectFactory objectFactory, AbstractBaseTable baseTbl,
     List<XFlattenedColumn> columnList, String chainName) {
-    for (ExprColumn expr : baseTbl.getExpressions()) {
-      XFlattenedColumn fcol = objectFactory.createXFlattenedColumn();
-      fcol.setExpression(JAXBUtils.xExprColumnFromHiveExprColumn(expr));
-      fcol.setTableName(baseTbl.getName());
-      if (chainName != null) {
-        fcol.setChainName(chainName);
+    if (baseTbl.getExpressions() != null) {
+      for (ExprColumn expr : baseTbl.getExpressions()) {
+        XFlattenedColumn fcol = objectFactory.createXFlattenedColumn();
+        fcol.setExpression(JAXBUtils.xExprColumnFromHiveExprColumn(expr));
+        fcol.setTableName(baseTbl.getName());
+        if (chainName != null) {
+          fcol.setChainName(chainName);
+        }
+        columnList.add(fcol);
+      }
+    }
+  }
+
+  private void addAllDirectExpressionsToFlattenedList(ObjectFactory objectFactory, CubeInterface baseTbl,
+    List<XFlattenedColumn> columnList, String chainName) {
+    if (baseTbl.getExpressions() != null) {
+      for (ExprColumn expr : baseTbl.getExpressions()) {
+        XFlattenedColumn fcol = objectFactory.createXFlattenedColumn();
+        fcol.setExpression(JAXBUtils.xExprColumnFromHiveExprColumn(expr));
+        fcol.setTableName(baseTbl.getName());
+        if (chainName != null) {
+          fcol.setChainName(chainName);
+        }
+        columnList.add(fcol);
       }
-      columnList.add(fcol);
     }
   }
 
@@ -1311,7 +1327,8 @@ public class CubeMetastoreServiceImpl extends LensService implements CubeMetasto
   }
 
   @Override
-  public XFlattenedColumns getFlattenedColumns(LensSessionHandle sessionHandle, String tableName) throws LensException {
+  public XFlattenedColumns getFlattenedColumns(LensSessionHandle sessionHandle, String tableName, boolean addChains)
+    throws LensException {
     try {
       acquire(sessionHandle);
       CubeMetastoreClient client = getClient(sessionHandle);
@@ -1324,13 +1341,17 @@ public class CubeMetastoreServiceImpl extends LensService implements CubeMetasto
         CubeInterface cube = client.getCube(tableName);
         addAllMeasuresToFlattenedList(objectFactory, cube, columnList);
         addAllDirectAttributesToFlattenedListFromCube(objectFactory, cube, columnList);
-        addAllDirectExpressionsToFlattenedList(objectFactory, (AbstractBaseTable) cube, columnList, null);
-        addAllChainedColsToFlattenedListFromCube(client, objectFactory, cube, columnList);
+        addAllDirectExpressionsToFlattenedList(objectFactory, cube, columnList, null);
+        if (addChains) {
+          addAllChainedColsToFlattenedListFromCube(client, objectFactory, cube, columnList);
+        }
       } else if (client.isDimension(tableName)) {
         Dimension dimension = client.getDimension(tableName);
         addAllDirectAttributesToFlattenedListFromDimension(objectFactory, dimension, columnList, null);
-        addAllDirectExpressionsToFlattenedList(objectFactory, (AbstractBaseTable) dimension, columnList, null);
-        addAllChainedColsToFlattenedList(client, objectFactory, dimension, columnList);
+        addAllDirectExpressionsToFlattenedList(objectFactory, dimension, columnList, null);
+        if (addChains) {
+          addAllChainedColsToFlattenedList(client, objectFactory, dimension, columnList);
+        }
       } else {
         throw new BadRequestException("Can't get reachable columns. '"
           + tableName + "' is neither a cube nor a dimension");
@@ -1370,4 +1391,33 @@ public class CubeMetastoreServiceImpl extends LensService implements CubeMetasto
     }
     return ret;
   }
+
+  @Override
+  public XJoinChains getAllJoinChains(LensSessionHandle sessionHandle, String tableName) throws LensException {
+    try {
+      acquire(sessionHandle);
+      CubeMetastoreClient client = getClient(sessionHandle);
+      Set<JoinChain> chains;
+      if (client.isCube(tableName)) {
+        chains = client.getCube(tableName).getJoinChains();
+      } else if (client.isDimension(tableName)) {
+        chains = client.getDimension(tableName).getJoinChains();
+      } else {
+        throw new BadRequestException("Can't get join chains. '"
+          + tableName + "' is neither a cube nor a dimension");
+      }
+      XJoinChains xJoinChains= new XJoinChains();
+      List<XJoinChain> joinChains = xJoinChains.getJoinChain();
+      if (chains != null) {
+        for (JoinChain chain : chains) {
+          joinChains.add(JAXBUtils.getXJoinChainFromJoinChain(chain));
+        }
+      }
+      return xJoinChains;
+    } catch (HiveException e) {
+      throw new LensException(e);
+    } finally {
+      release(sessionHandle);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/ff31ad96/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java b/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java
index e746ebb..130ba3e 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java
@@ -24,11 +24,8 @@ import javax.ws.rs.*;
 import javax.ws.rs.core.MediaType;
 import javax.xml.bind.JAXBElement;
 
-import org.apache.lens.api.APIResult;
+import org.apache.lens.api.*;
 import org.apache.lens.api.APIResult.Status;
-import org.apache.lens.api.DateTime;
-import org.apache.lens.api.LensSessionHandle;
-import org.apache.lens.api.StringList;
 import org.apache.lens.api.metastore.*;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.error.LensException;
@@ -1515,24 +1512,47 @@ public class MetastoreResource {
   /**
    * Get flattened list of columns reachable from a cube or a dimension
    *
-   * @param sessionid session id
-   * @param tableName name of the table
+   * @param sessionid  session id
+   * @param tableName  name of the table
+   * @param addChains whether columns accessed via chains should also be returned
    * @return list of measures, expressions or dimension attributes
    */
   @GET
   @Path("flattened/{tableName}")
   public JAXBElement<XFlattenedColumns> getFlattenedColumns(
     @QueryParam("sessionid") LensSessionHandle sessionid,
-    @PathParam("tableName") String tableName) {
+    @PathParam("tableName") String tableName, @QueryParam("add_chains") @DefaultValue("true") boolean addChains) {
     checkSessionId(sessionid);
     try {
-      return X_CUBE_OBJECT_FACTORY.createXFlattenedColumns(getSvc().getFlattenedColumns(sessionid, tableName));
+      return X_CUBE_OBJECT_FACTORY.createXFlattenedColumns(
+        getSvc().getFlattenedColumns(sessionid, tableName, addChains));
     } catch (LensException exc) {
       throw new WebApplicationException(exc);
     }
   }
 
   /**
+   * Get all chains that belong to a table(cube or dimension) in the metastore
+   *
+   * @param sessionid The sessionid in which user is working
+   * @param tableName name of the table. can be either cube or dimension
+   * @return {@link XJoinChains} object
+   */
+  @GET
+  @Path("/chains/{tableName}")
+  public JAXBElement<XJoinChains> getAllJoinChains(
+    @QueryParam("sessionid") LensSessionHandle sessionid, @PathParam("tableName") String tableName)
+    throws LensException {
+    checkSessionId(sessionid);
+    try {
+      return X_CUBE_OBJECT_FACTORY.createXJoinChains(getSvc().getAllJoinChains(sessionid, tableName));
+    } catch (LensException exc) {
+      checkTableNotFound(exc, tableName);
+      throw exc;
+    }
+  }
+
+  /**
    * Get the latest available date upto which data is available for the base cubes, for the time dimension.
    *
    * @param sessionid     The sessionid in which user is working

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/ff31ad96/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e2cbac5..6335ca5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -71,7 +71,7 @@
     <metrics.version>3.0.2</metrics.version>
     <subethasmtp.version>3.1.7</subethasmtp.version>
     <liquibase.version>3.0.7</liquibase.version>
-    <spring.shell.version>1.0.0.RELEASE</spring.shell.version>
+    <spring.shell.version>1.1.0.RELEASE</spring.shell.version>
 
     <c3p0.version>0.9.5</c3p0.version>
     <hsqldb.version>2.2.9</hsqldb.version>
@@ -909,6 +909,11 @@
         <version>${typesafe.config.version}</version>
       </dependency>
       <dependency>
+        <groupId>org.jvnet.jaxb2_commons</groupId>
+        <artifactId>jaxb2-basics-runtime</artifactId>
+        <version>${jaxb2.basics.plugin.version}</version>
+      </dependency>
+      <dependency>
         <groupId>org.apache.hive</groupId>
         <artifactId>hive-shims</artifactId>
         <version>${hive.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/ff31ad96/src/site/apt/user/cli.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/user/cli.apt b/src/site/apt/user/cli.apt
index a5bc056..20785f1 100644
--- a/src/site/apt/user/cli.apt
+++ b/src/site/apt/user/cli.apt
@@ -157,7 +157,11 @@ User CLI Commands
 *--+--+
 |create cube [--path] \<path-to-cube-spec-file\>|Create a new Cube, taking spec from <<<path-to-cube-spec-file>>>|
 *--+--+
-|cube latestdate [--cube] \<cube_name\> [--time_dimension] \<time_dimension\>|get latest date of data available in cube <<<cube_name>>> for time dimension <<<time_dimension_name>>>.  Instead of time dimension, partition column can be directly passed as <<<time_dimension>>>|
+|cube latestdate [--name] \<cube_name\> [--time_dimension] \<time_dimension\>|get latest date of data available in cube <<<cube_name>>> for time dimension <<<time_dimension_name>>>.  Instead of time dimension, partition column can be directly passed as <<<time_dimension>>>|
+*--+--+
+|cube show fields [--name] \<cube_name\> [--flattened \<flattened\>]|Show queryable fields of the given cube <<<cube_name>>>. Optionally specify <<<flattened>>> to include chained fields|
+*--+--+
+|cube show joinchains [--name] \<cube_name\>|Show joinchains of the given cube <<<cube_name>>>. |
 *--+--+
 |describe cube [--name] \<cube_name\>|describe cube with name <<<cube_name>>>|
 *--+--+
@@ -182,6 +186,10 @@ User CLI Commands
 *--+--+
 |describe dimension [--name] \<dimension_name\>|describe dimension <<<dimension_name>>>|
 *--+--+
+|dimension show fields [--name] \<dimension_name\> [--flattened \<flattened\>]|Show queryable fields of the given dimension <<<dimension_name>>>. Optionally specify <<<flattened>>> to include chained fields|
+*--+--+
+|dimension show joinchains [--name] \<dimension_name\>|Show joinchains of the given dimension <<<dimension_name>>>. |
+*--+--+
 |drop dimension [--name] \<dimension_name\>|drop dimension <<<dimension_name>>>|
 *--+--+
 |show dimensions|show list of all dimensions in current database|