You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by jo...@apache.org on 2019/04/04 01:14:20 UTC

[incubator-druid] branch 0.14.0-incubating updated: Add missing redirects and fix broken links (#7213) (#7409)

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

jonwei pushed a commit to branch 0.14.0-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/0.14.0-incubating by this push:
     new 0a1345e  Add missing redirects and fix broken links (#7213) (#7409)
0a1345e is described below

commit 0a1345e83153ccc791a8404ab3fb9e57cde4f320
Author: Jonathan Wei <jo...@users.noreply.github.com>
AuthorDate: Wed Apr 3 18:14:11 2019 -0700

    Add missing redirects and fix broken links (#7213) (#7409)
    
    * Add missing redirects
    
    * Fix zookeeper redirect
    
    * Fix broken links
---
 docs/_bin/make-redirects.py                        |  1 +
 docs/_bin/missing-redirect-finder.py               | 54 ++++++++++++++++++++++
 docs/_bin/missing-redirect-finder2.py              | 49 ++++++++++++++++++++
 docs/_redirects.json                               | 45 +++++++++++++++++-
 docs/content/configuration/index.md                |  4 +-
 docs/content/design/broker.md                      |  2 +-
 docs/content/design/coordinator.md                 |  2 +-
 docs/content/design/overlord.md                    |  2 +-
 .../development/extensions-core/bloom-filter.md    |  4 +-
 .../extensions-core/kinesis-ingestion.md           |  9 ++--
 .../extensions-core/simple-client-sslcontext.md    |  4 +-
 docs/content/ingestion/reports.md                  |  6 +--
 docs/content/operations/tls-support.md             |  6 +--
 docs/content/querying/aggregations.md              |  6 +--
 docs/content/querying/sql.md                       |  8 ++--
 15 files changed, 175 insertions(+), 27 deletions(-)

diff --git a/docs/_bin/make-redirects.py b/docs/_bin/make-redirects.py
index 5a32a0d..5affcb5 100755
--- a/docs/_bin/make-redirects.py
+++ b/docs/_bin/make-redirects.py
@@ -57,6 +57,7 @@ for redirect in redirects:
       raise Exception('Redirect target does not exist for source: ' + source)
 
   # Write redirect file
+  os.makedirs(os.path.dirname(source_file), exist_ok=True)
   with open(source_file, 'w') as f:
     f.write("---\n")
     f.write("layout: redirect_page\n")
diff --git a/docs/_bin/missing-redirect-finder.py b/docs/_bin/missing-redirect-finder.py
new file mode 100755
index 0000000..732cd80
--- /dev/null
+++ b/docs/_bin/missing-redirect-finder.py
@@ -0,0 +1,54 @@
+#!/usr/bin/env python3
+
+# 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.
+
+import os
+import subprocess
+import sys
+
+deleted_paths_dict = {}
+
+# assumes docs/latest in the doc repo has the current files for the next release
+# deletes docs for old versions and copies docs/latest into the old versions
+# run `git status | grep deleted:` on the doc repo to see what pages were deleted and feed that into
+# missing-redirect-finder2.py
+def main():
+    if len(sys.argv) != 2:
+      sys.stderr.write('usage: program <druid-docs-repo-path>\n')
+      sys.exit(1)
+
+    druid_docs_path = sys.argv[1]
+    druid_docs_path = "{}/docs".format(druid_docs_path)
+    prev_release_doc_paths = os.listdir(druid_docs_path)
+    for doc_path in prev_release_doc_paths:
+        if (doc_path != "img" and doc_path != "latest"):
+            print("DOC PATH: " + doc_path)
+
+            try:
+                command = "rm -rf {}/{}/*".format(druid_docs_path, doc_path)
+                outstr = subprocess.check_output(command, shell=True).decode('UTF-8')
+
+                command = "cp -r {}/latest/* {}/{}/".format(druid_docs_path, druid_docs_path, doc_path)
+                outstr = subprocess.check_output(command, shell=True).decode('UTF-8')
+            except:
+                print("error in path: " + doc_path)
+                continue
+
+if __name__ == "__main__":
+    try:
+        main()
+    except KeyboardInterrupt:
+        print('Interrupted, closing.')
diff --git a/docs/_bin/missing-redirect-finder2.py b/docs/_bin/missing-redirect-finder2.py
new file mode 100755
index 0000000..fd38393
--- /dev/null
+++ b/docs/_bin/missing-redirect-finder2.py
@@ -0,0 +1,49 @@
+#!/usr/bin/env python3
+
+# 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.
+
+import json
+import sys
+
+# Takes the output of `git status | grep deleted:` on the doc repo
+# and cross references deleted pages with the _redirects.json file
+if len(sys.argv) != 3:
+  sys.stderr.write('usage: program <del_paths_file> <redirect.json file>\n')
+  sys.exit(1)
+
+del_paths = sys.argv[1]
+redirect_json_path = sys.argv[2]
+
+dep_dict = {}
+with open(del_paths, 'r') as del_paths_file:
+    for line in del_paths_file.readlines():
+        subidx = line.index("/", 0)
+        line2 = line[subidx+1:]
+        subidx = line2.index("/", 0)
+        line3 = line2[subidx+1:]
+        dep_dict[line3.strip("\n")] = True
+
+existing_redirects = {}
+with open(redirect_json_path, 'r') as redirect_json_file:
+    redirect_json = json.load(redirect_json_file)
+    for redirect_entry in redirect_json:
+        redirect_source = redirect_entry["source"]
+        redirect_source = redirect_source.replace(".html", ".md")
+        existing_redirects[redirect_source] = True
+
+for dep in dep_dict:
+    if dep not in existing_redirects:
+        print("MISSING REDIRECT: " + dep)
diff --git a/docs/_redirects.json b/docs/_redirects.json
index 9e53a74..222c665 100644
--- a/docs/_redirects.json
+++ b/docs/_redirects.json
@@ -122,5 +122,48 @@
   {"source": "tutorials/tutorial-loading-batch-data.html", "target": "tutorial-batch.html"},
   {"source": "tutorials/tutorial-loading-streaming-data.html", "target": "tutorial-streams.html"},
   {"source": "tutorials/tutorial-the-druid-cluster.html", "target": "cluster.html"},
-  {"source": "development/extensions-core/caffeine-cache.html", "target":"../../configuration/caching.html"}
+  {"source": "development/extensions-core/caffeine-cache.html", "target":"../../configuration/caching.html"},
+  {"source": "Production-Cluster-Configuration.html", "target": "tutorials/cluster.html"},
+  {"source": "development/extensions-contrib/parquet.html", "target":"../../development/extensions-core/parquet.html"},
+  {"source": "development/extensions-contrib/scan-query.html", "target":"../../querying/scan-query.html"},
+  {"source": "tutorials/ingestion.html", "target": "index.html"},
+  {"source": "tutorials/ingestion-streams.html", "target": "index.html"},
+  {"source": "ingestion/native-batch.html", "target": "native_tasks.html"},
+  {"source": "Compute.html", "target": "design/processes.html"},
+  {"source": "Contribute.html", "target": "../../community/index.html"},
+  {"source": "Download.html", "target": "../../downloads.html"},
+  {"source": "Druid-Personal-Demo-Cluster.html", "target": "tutorials/index.html"},
+  {"source": "Home.html", "target": "index.html"},
+  {"source": "Loading-Your-Data.html", "target": "ingestion/index.html"},
+  {"source": "Master.html", "target": "design/processes.html"},
+  {"source": "MySQL.html", "target": "development/extensions-core/mysql.html"},
+  {"source": "OrderBy.html", "target": "querying/limitspec.html"},
+  {"source": "Querying-your-data.html", "target": "querying/querying.html"},
+  {"source": "Spatial-Filters.html", "target": "development/geo.html"},
+  {"source": "Spatial-Indexing.html", "target": "development/geo.html"},
+  {"source": "Stand-Alone-With-Riak-CS.html", "target": "index.html"},
+  {"source": "Support.html", "target": "../../community/index.html"},
+  {"source": "Tutorial:-Webstream.html", "target": "tutorials/index.html"},
+  {"source": "Twitter-Tutorial.html", "target": "tutorials/index.html"},
+  {"source": "Tutorial:-Loading-Your-Data-Part-1.html", "target": "tutorials/index.html"},
+  {"source": "Tutorial:-Loading-Your-Data-Part-2.html", "target": "tutorials/index.html"},
+  {"source": "Kafka-Eight.html", "target": "development/extensions-core/kafka-eight-firehose.html"},
+  {"source": "Thanks.html", "target": "../../community/index.html"},
+  {"source": "Tutorial-A-First-Look-at-Druid.html", "target": "tutorials/index.html"},
+  {"source": "Tutorial-All-About-Queries.html", "target": "tutorials/index.html"},
+  {"source": "Tutorial-Loading-Batch-Data.html", "target": "tutorials/index.html"},
+  {"source": "Tutorial-Loading-Streaming-Data.html", "target": "tutorials/index.html"},
+  {"source": "Tutorial-The-Druid-Cluster.html", "target": "tutorials/index.html"},
+  {"source": "configuration/hadoop.html", "target": "ingestion/hadoop.html"},
+  {"source": "configuration/production-cluster.html", "target": "tutorials/cluster.html"},
+  {"source": "configuration/zookeeper.html", "target": "dependencies/zookeeper.html"},
+  {"source": "querying/optimizations.html", "target": "dependencies/cluster.html"},
+  {"source": "development/community-extensions/azure.html", "target": "../extensions-contrib/azure.html"},
+  {"source": "development/community-extensions/cassandra.html", "target": "../extensions-contrib/cassandra.html"},
+  {"source": "development/community-extensions/cloudfiles.html", "target": "../extensions-contrib/cloudfiles.html"},
+  {"source": "development/community-extensions/graphite.html", "target": "../extensions-contrib/graphite.html"},
+  {"source": "development/community-extensions/kafka-simple.html", "target": "../extensions-contrib/kafka-simple.html"},
+  {"source": "development/community-extensions/rabbitmq.html", "target": "../extensions-contrib/rabbitmq.html"},
+  {"source": "development/extensions-core/namespaced-lookup.html", "target": "lookups-cached-global.html"},
+  {"source": "operations/insert-segment-to-db.html", "target": "../index.html"}
 ]
diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md
index 712e1b7..ac703c8 100644
--- a/docs/content/configuration/index.md
+++ b/docs/content/configuration/index.md
@@ -243,9 +243,9 @@ and `druid.tlsPort` properties on each process. Please see `Configuration` secti
 #### Jetty Server TLS Configuration
 
 Druid uses Jetty as an embedded web server. To get familiar with TLS/SSL in general and related concepts like Certificates etc.
-reading this [Jetty documentation](http://www.eclipse.org/jetty/documentation/9.3.x/configuring-ssl.html) might be helpful.
+reading this [Jetty documentation](http://www.eclipse.org/jetty/documentation/9.4.x/configuring-ssl.html) might be helpful.
 To get more in depth knowledge of TLS/SSL support in Java in general, please refer to this [guide](http://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html).
-The documentation [here](http://www.eclipse.org/jetty/documentation/9.3.x/configuring-ssl.html#configuring-sslcontextfactory)
+The documentation [here](http://www.eclipse.org/jetty/documentation/9.4.x/configuring-ssl.html#configuring-sslcontextfactory)
 can help in understanding TLS/SSL configurations listed below. This [document](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html) lists all the possible
 values for the below mentioned configs among others provided by Java implementation.
 
diff --git a/docs/content/design/broker.md b/docs/content/design/broker.md
index fe9119d..29ea42e 100644
--- a/docs/content/design/broker.md
+++ b/docs/content/design/broker.md
@@ -47,7 +47,7 @@ org.apache.druid.cli.Main server broker
 
 Most druid queries contain an interval object that indicates a span of time for which data is requested. Likewise, Druid [Segments](../design/segments.html) are partitioned to contain data for some interval of time and segments are distributed across a cluster. Consider a simple datasource with 7 segments where each segment contains data for a given day of the week. Any query issued to the datasource for more than one day of data will hit more than one segment. These segments will likely [...]
 
-To determine which processes to forward queries to, the Broker process first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [Historical](../design/historical.html) and streaming ingestion [Peon](../design/peon.html) processes and the segments they are serving. For every datasource in Zookeeper, the Broker process builds a timeline of segments and the processes that serve them. When queries are received for a specific datasource and interva [...]
+To determine which processes to forward queries to, the Broker process first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [Historical](../design/historical.html) and streaming ingestion [Peon](../design/peons.html) processes and the segments they are serving. For every datasource in Zookeeper, the Broker process builds a timeline of segments and the processes that serve them. When queries are received for a specific datasource and interv [...]
 
 ### Caching
 
diff --git a/docs/content/design/coordinator.md b/docs/content/design/coordinator.md
index 9571f3a..810f212 100644
--- a/docs/content/design/coordinator.md
+++ b/docs/content/design/coordinator.md
@@ -113,7 +113,7 @@ If it finds such segments, it simply skips them.
 
 ### The Coordinator Console
 
-The Druid Coordinator exposes a web GUI for displaying cluster information and rule configuration. For more details, please see [coordinator console](../operations/web-consoles.html#coordinator-console).
+The Druid Coordinator exposes a web GUI for displaying cluster information and rule configuration. For more details, please see [coordinator console](../operations/management-uis.html#coordinator-consoles).
 
 ### FAQ
 
diff --git a/docs/content/design/overlord.md b/docs/content/design/overlord.md
index 40f3c44..9fb2465 100644
--- a/docs/content/design/overlord.md
+++ b/docs/content/design/overlord.md
@@ -41,7 +41,7 @@ This mode is recommended if you intend to use the indexing service as the single
 
 ### Overlord Console
 
-The Overlord provides a UI for managing tasks and workers. For more details, please see [overlord console](../operations/web-consoles.html#overlord-console).
+The Overlord provides a UI for managing tasks and workers. For more details, please see [overlord console](../operations/management-uis.html#overlord-console).
 
 ### Blacklisted Workers
 
diff --git a/docs/content/development/extensions-core/bloom-filter.md b/docs/content/development/extensions-core/bloom-filter.md
index 651cc30..86c645d 100644
--- a/docs/content/development/extensions-core/bloom-filter.md
+++ b/docs/content/development/extensions-core/bloom-filter.md
@@ -76,7 +76,7 @@ This string can then be used in the native or sql Druid query.
 |`type`                   |Filter Type. Should always be `bloom`|yes|
 |`dimension`              |The dimension to filter over. | yes |
 |`bloomKFilter`           |Base64 encoded Binary representation of `org.apache.hive.common.util.BloomKFilter`| yes |
-|`extractionFn`|[Extraction function](./../dimensionspecs.html#extraction-functions) to apply to the dimension values |no|
+|`extractionFn`|[Extraction function](../../querying/dimensionspecs.html#extraction-functions) to apply to the dimension values |no|
 
 
 ### Serialized Format for BloomKFilter
@@ -129,7 +129,7 @@ for the query.
 |-------------------------|------------------------------|----------------------------------|
 |`type`                   |Aggregator Type. Should always be `bloom`|yes|
 |`name`                   |Output field name |yes|
-|`field`                  |[DimensionSpec](./../dimensionspecs.html) to add to `org.apache.hive.common.util.BloomKFilter` | yes |
+|`field`                  |[DimensionSpec](../../querying/dimensionspecs.html) to add to `org.apache.hive.common.util.BloomKFilter` | yes |
 |`maxNumEntries`          |Maximum number of distinct values supported by `org.apache.hive.common.util.BloomKFilter`, default `1500`| no |
 
 ### Example
diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md
index 4704170..38e3830 100644
--- a/docs/content/development/extensions-core/kinesis-ingestion.md
+++ b/docs/content/development/extensions-core/kinesis-ingestion.md
@@ -1,3 +1,8 @@
+---
+layout: doc_page
+title: "Kinesis Indexing Service"
+---
+
 <!--
   ~ Licensed to the Apache Software Foundation (ASF) under one
   ~ or more contributor license agreements.  See the NOTICE file
@@ -17,10 +22,6 @@
   ~ under the License.
   -->
 
----
-layout: doc_page
----
-
 # Kinesis Indexing Service
 
 Similar to the [Kafka indexing service](./kafka-ingestion.html), the Kinesis indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from
diff --git a/docs/content/development/extensions-core/simple-client-sslcontext.md b/docs/content/development/extensions-core/simple-client-sslcontext.md
index 107739e..9c2638f 100644
--- a/docs/content/development/extensions-core/simple-client-sslcontext.md
+++ b/docs/content/development/extensions-core/simple-client-sslcontext.md
@@ -45,9 +45,9 @@ The following table contains optional parameters for supporting client certifica
 |`druid.client.https.keyStorePath`|The file path or URL of the TLS/SSL Key store containing the client certificate that Druid will use when communicating with other Druid services. If this is null, the other properties in this table are ignored.|none|yes|
 |`druid.client.https.keyStoreType`|The type of the key store.|none|yes|
 |`druid.client.https.certAlias`|Alias of TLS client certificate in the keystore.|none|yes|
-|`druid.client.https.keyStorePassword`|The [Password Provider](../operations/password-provider.html) or String password for the Key Store.|none|no|
+|`druid.client.https.keyStorePassword`|The [Password Provider](../../operations/password-provider.html) or String password for the Key Store.|none|no|
 |`druid.client.https.keyManagerFactoryAlgorithm`|Algorithm to use for creating KeyManager, more details [here](https://docs.oracle.com/javase/7/docs/technotes/guides/security/jsse/JSSERefGuide.html#KeyManager).|`javax.net.ssl.KeyManagerFactory.getDefaultAlgorithm()`|no|
-|`druid.client.https.keyManagerPassword`|The [Password Provider](../operations/password-provider.html) or String password for the Key Manager.|none|no|
+|`druid.client.https.keyManagerPassword`|The [Password Provider](../../operations/password-provider.html) or String password for the Key Manager.|none|no|
 |`druid.client.https.validateHostnames`|Validate the hostname of the server. This should not be disabled unless you are using [custom TLS certificate checks](../../operations/tls-support.html#custom-tls-certificate-checks) and know that standard hostname validation is not needed.|true|no|
 
 This [document](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html) lists all the possible
diff --git a/docs/content/ingestion/reports.md b/docs/content/ingestion/reports.md
index 8696374..922fa77 100644
--- a/docs/content/ingestion/reports.md
+++ b/docs/content/ingestion/reports.md
@@ -88,7 +88,7 @@ The `errorMsg` field shows a message describing the error that caused a task to
 
 ### Row stats
 
-The non-parallel [Native Batch Task](../native_tasks.md), the Hadoop batch task, and the tasks created by the Kafka Indexing Service support retrieval of row stats while the task is running.
+The non-parallel [Native Batch Task](../ingestion/native_tasks.html), the Hadoop batch task, and the tasks created by the Kafka Indexing Service support retrieval of row stats while the task is running.
 
 The live report can be accessed with a GET to the following URL on a Peon running a task:
 
@@ -133,7 +133,7 @@ An example report is shown below. The `movingAverages` section contains 1 minute
 }
 ```
 
-Note that this is only supported by the non-parallel [Native Batch Task](../native_tasks.md), the Hadoop Batch task, and the tasks created by the Kafka Indexing Service.
+Note that this is only supported by the non-parallel [Native Batch Task](../ingestion/native_tasks.html), the Hadoop Batch task, and the tasks created by the Kafka Indexing Service.
 
 For the Kafka Indexing Service, a GET to the following Overlord API will retrieve live row stat reports from each task being managed by the supervisor and provide a combined report.
 
@@ -149,4 +149,4 @@ Current lists of unparseable events can be retrieved from a running task with a
 http://<middlemanager-host>:<worker-port>/druid/worker/v1/chat/<task-id>/unparseableEvents
 ```
 
-Note that this is only supported by the non-parallel [Native Batch Task](../native_tasks.md) and the tasks created by the Kafka Indexing Service.
+Note that this is only supported by the non-parallel [Native Batch Task](../ingestion/native_tasks.html) and the tasks created by the Kafka Indexing Service.
diff --git a/docs/content/operations/tls-support.md b/docs/content/operations/tls-support.md
index 5585e62..744f20c 100644
--- a/docs/content/operations/tls-support.md
+++ b/docs/content/operations/tls-support.md
@@ -37,9 +37,9 @@ and `druid.tlsPort` properties on each process. Please see `Configuration` secti
 # Jetty Server TLS Configuration
 
 Druid uses Jetty as an embedded web server. To get familiar with TLS/SSL in general and related concepts like Certificates etc.
-reading this [Jetty documentation](http://www.eclipse.org/jetty/documentation/9.3.x/configuring-ssl.html) might be helpful.
+reading this [Jetty documentation](http://www.eclipse.org/jetty/documentation/9.4.x/configuring-ssl.html) might be helpful.
 To get more in depth knowledge of TLS/SSL support in Java in general, please refer to this [guide](http://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html).
-The documentation [here](http://www.eclipse.org/jetty/documentation/9.3.x/configuring-ssl.html#configuring-sslcontextfactory)
+The documentation [here](http://www.eclipse.org/jetty/documentation/9.4.x/configuring-ssl.html#configuring-sslcontextfactory)
 can help in understanding TLS/SSL configurations listed below. This [document](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html) lists all the possible
 values for the below mentioned configs among others provided by Java implementation.
 
@@ -58,7 +58,7 @@ The following table contains configuration options related to client certificate
 |`druid.server.https.trustStoreType`|The type of the trust store containing certificates used to validate client certificates. Not needed if `requireClientCertificate` is false.|`java.security.KeyStore.getDefaultType()`|no|
 |`druid.server.https.trustStorePath`|The file path or URL of the trust store containing certificates used to validate client certificates. Not needed if `requireClientCertificate` is false.|none|yes, only if `requireClientCertificate` is true|
 |`druid.server.https.trustStoreAlgorithm`|Algorithm to be used by TrustManager to validate client certificate chains. Not needed if `requireClientCertificate` is false.|`javax.net.ssl.TrustManagerFactory.getDefaultAlgorithm()`|no|
-|`druid.server.https.trustStorePassword`|The [Password Provider](../../operations/password-provider.html) or String password for the Trust Store.  Not needed if `requireClientCertificate` is false.|none|no|
+|`druid.server.https.trustStorePassword`|The [Password Provider](../operations/password-provider.html) or String password for the Trust Store.  Not needed if `requireClientCertificate` is false.|none|no|
 |`druid.server.https.validateHostnames`|If set to true, check that the client's hostname matches the CN/subjectAltNames in the client certificate.  Not used if `requireClientCertificate` is false.|true|no|
 |`druid.server.https.crlPath`|Specifies a path to a file containing static [Certificate Revocation Lists](https://en.wikipedia.org/wiki/Certificate_revocation_list), used to check if a client certificate has been revoked. Not used if `requireClientCertificate` is false.|null|no|
 
diff --git a/docs/content/querying/aggregations.md b/docs/content/querying/aggregations.md
index a9a819b..ce69b6d 100644
--- a/docs/content/querying/aggregations.md
+++ b/docs/content/querying/aggregations.md
@@ -280,10 +280,10 @@ The [DataSketches HLL Sketch](../development/extensions-core/datasketches-hll.ht
 #### Cardinality/HyperUnique (Deprecated)
 
 <div class="note caution">
-The Cardinality and HyperUnique aggregators are deprecated. Please use <a href="../extensions-core/datasketches-hll.html">DataSketches HLL Sketch</a> instead.
+The Cardinality and HyperUnique aggregators are deprecated. Please use <a href="../development/extensions-core/datasketches-hll.html">DataSketches HLL Sketch</a> instead.
 </div>
 
-The [Cardinality and HyperUnique](../hll-old.html) aggregators are older aggregator implementations available by default in Druid that also provide distinct count estimates using the HyperLogLog algorithm. The newer [DataSketches HLL Sketch](../development/extensions-core/datasketches-hll.html) extension-provided aggregator has superior accuracy and performance and is recommended instead. 
+The [Cardinality and HyperUnique](../querying/hll-old.html) aggregators are older aggregator implementations available by default in Druid that also provide distinct count estimates using the HyperLogLog algorithm. The newer [DataSketches HLL Sketch](../development/extensions-core/datasketches-hll.html) extension-provided aggregator has superior accuracy and performance and is recommended instead. 
 
 The DataSketches team has published a [comparison study](https://datasketches.github.io/docs/HLL/HllSketchVsDruidHyperLogLogCollector.html) between Druid's original HLL algorithm and the DataSketches HLL algorithm. Based on the demonstrated advantages of the DataSketches implementation, we have deprecated Druid's original HLL aggregator.
 
@@ -307,7 +307,7 @@ We do not recommend the fixed buckets histogram for general use, as its usefulne
 
 #### Approximate Histogram (Deprecated)
 
-The [Approximate Histogram](../development/extensions-core/approxiate-histograms.html) extension-provided aggregator also provides quantile estimates and histogram approximations, based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf).
+The [Approximate Histogram](../development/extensions-core/approximate-histograms.html) extension-provided aggregator also provides quantile estimates and histogram approximations, based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf).
 
 The algorithm used by this deprecated aggregator is highly distribution-dependent and its output is subject to serious distortions when the input does not fit within the algorithm's limitations.
 
diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md
index 31e0109..288bfb0 100644
--- a/docs/content/querying/sql.md
+++ b/docs/content/querying/sql.md
@@ -119,10 +119,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`MAX(expr)`|Takes the maximum of numbers.|
 |`AVG(expr)`|Averages numbers.|
 |`APPROX_COUNT_DISTINCT(expr)`|Counts distinct values of expr, which can be a regular column or a hyperUnique column. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`.|
-|`APPROX_COUNT_DISTINCT_DS_HLL(expr, [lgK, tgtHllType])`|Counts distinct values of expr, which can be a regular column or an [HLL sketch](../development/extensions-core/datasketches-hll.html) column. The `lgK` and `tgtHllType` parameters are described in the HLL sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extensions.html [...]
-|`APPROX_COUNT_DISTINCT_DS_THETA(expr, [size])`|Counts distinct values of expr, which can be a regular column or a [Theta sketch](../development/extensions-core/datasketches-theta.html) column. The `size` parameter is described in the Theta sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extensions.html) must be loaded to us [...]
+|`APPROX_COUNT_DISTINCT_DS_HLL(expr, [lgK, tgtHllType])`|Counts distinct values of expr, which can be a regular column or an [HLL sketch](../development/extensions-core/datasketches-hll.html) column. The `lgK` and `tgtHllType` parameters are described in the HLL sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extension.html) [...]
+|`APPROX_COUNT_DISTINCT_DS_THETA(expr, [size])`|Counts distinct values of expr, which can be a regular column or a [Theta sketch](../development/extensions-core/datasketches-theta.html) column. The `size` parameter is described in the Theta sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extension.html) must be loaded to use [...]
 |`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or [approxHistogram](../development/extensions-core/approximate-histograms.html#approximate-histogram-aggregator) exprs. The "probability" should be between 0 and 1 (exclusive). The "resolution" is the number of centroids to use for the computation. Higher resolutions will give more precise results but also have higher overhead. If not provided, the default resolution is 50. The [approximate his [...]
-|`APPROX_QUANTILE_DS(expr, probability, [k])`|Computes approximate quantiles on numeric or [Quantiles sketch](../development/extensions-core/datasketches-quantiles.html) exprs. The "probability" should be between 0 and 1 (exclusive). The `k` parameter is described in the Quantiles sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extensions.html) must be loaded to use this function.|
+|`APPROX_QUANTILE_DS(expr, probability, [k])`|Computes approximate quantiles on numeric or [Quantiles sketch](../development/extensions-core/datasketches-quantiles.html) exprs. The "probability" should be between 0 and 1 (exclusive). The `k` parameter is described in the Quantiles sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.html) must be loaded to use this function.|
 |`APPROX_QUANTILE_FIXED_BUCKETS(expr, probability, numBuckets, lowerLimit, upperLimit, [outlierHandlingMode])`|Computes approximate quantiles on numeric or [fixed buckets histogram](../development/extensions-core/approximate-histograms.html#fixed-buckets-histogram) exprs. The "probability" should be between 0 and 1 (exclusive). The `numBuckets`, `lowerLimit`, `upperLimit`, and `outlierHandlingMode` parameters are described in the fixed buckets histogram documentation. The [approximate hi [...]
 |`BLOOM_FILTER(expr, numEntries)`|Computes a bloom filter from values produced by `expr`, with `numEntries` maximum number of distinct values before false positve rate increases. See [bloom filter extension](../development/extensions-core/bloom-filter.html) documentation for additional details.|
 
@@ -660,7 +660,7 @@ check out [ingestion tasks](#../ingestion/tasks.html)
 |Column|Notes|
 |------|-----|
 |task_id|Unique task identifier|
-|type|Task type, for example this value is "index" for indexing tasks. See [tasks-overview](../ingestion/tasks.md)|
+|type|Task type, for example this value is "index" for indexing tasks. See [tasks-overview](../ingestion/tasks.html)|
 |datasource|Datasource name being indexed|
 |created_time|Timestamp in ISO8601 format corresponding to when the ingestion task was created. Note that this value is populated for completed and waiting tasks. For running and pending tasks this value is set to 1970-01-01T00:00:00Z|
 |queue_insertion_time|Timestamp in ISO8601 format corresponding to when this task was added to the queue on the Overlord|


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